From 6dd0c0a90b1c2aaa56e3f144ecc57c996715c481 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 1 Jun 2015 17:58:08 +0900 Subject: [PATCH 01/80] merge from TAJO-1615 --- .../physical/HashShuffleFileWriteExec.java | 2 +- .../engine/planner/physical/PhysicalExec.java | 2 +- .../tajo/master/rm/TajoResourceTracker.java | 4 +- .../util/TajoUncaughtExceptionHandler.java | 70 ++ .../tajo/util/history/HistoryWriter.java | 2 +- .../tajo/worker/ExecutionBlockContext.java | 83 +- .../apache/tajo/worker/LegacyTaskImpl.java | 844 ++++++++++++++++++ .../tajo/worker/NodeResourceManager.java | 45 +- .../apache/tajo/worker/NodeStatusUpdater.java | 34 +- .../org/apache/tajo/worker/TajoWorker.java | 47 +- .../tajo/worker/TajoWorkerManagerService.java | 9 +- .../java/org/apache/tajo/worker/Task.java | 834 +---------------- .../tajo/worker/TaskAttemptContext.java | 58 +- .../org/apache/tajo/worker/TaskContainer.java | 85 ++ .../org/apache/tajo/worker/TaskExecutor.java | 194 ++++ .../java/org/apache/tajo/worker/TaskImpl.java | 838 +++++++++++++++++ .../org/apache/tajo/worker/TaskManager.java | 180 ++++ .../org/apache/tajo/worker/TaskRunner.java | 10 +- .../apache/tajo/worker/TaskRunnerHistory.java | 1 + .../apache/tajo/worker/TaskRunnerManager.java | 12 +- .../event/ExecutionBlockStartEvent.java | 35 + .../worker/event/ExecutionBlockStopEvent.java | 37 + .../event/NodeResourceAllocateEvent.java | 2 +- .../event/NodeResourceDeallocateEvent.java | 2 +- ...nagerEvent.java => NodeResourceEvent.java} | 9 +- .../tajo/worker/event/NodeStatusEvent.java | 11 +- .../tajo/worker/event/TaskExecutorEvent.java | 44 + .../tajo/worker/event/TaskManagerEvent.java | 43 + .../tajo/worker/event/TaskRunnerEvent.java | 1 + .../worker/event/TaskRunnerStartEvent.java | 44 +- .../worker/event/TaskRunnerStopEvent.java | 1 + .../tajo/worker/event/TaskStartEvent.java | 44 + .../src/main/proto/TajoWorkerProtocol.proto | 1 + .../tajo/querymaster/TestKillQuery.java | 135 ++- .../tajo/worker/MockExecutionBlock.java | 42 + .../tajo/worker/MockNodeResourceManager.java | 96 ++ .../tajo/worker/MockNodeStatusUpdater.java | 4 +- .../apache/tajo/worker/MockTaskExecutor.java | 141 +++ .../apache/tajo/worker/MockTaskManager.java | 59 ++ .../apache/tajo/worker/MockWorkerContext.java | 129 +++ .../org/apache/tajo/worker/TestFetcher.java | 14 +- .../tajo/worker/TestNodeResourceManager.java | 135 ++- .../tajo/worker/TestNodeStatusUpdater.java | 54 +- .../apache/tajo/worker/TestTaskExecutor.java | 330 +++++++ .../apache/tajo/worker/TestTaskManager.java | 185 ++++ 45 files changed, 3870 insertions(+), 1082 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java rename tajo-core/src/main/java/org/apache/tajo/worker/event/{NodeResourceManagerEvent.java => NodeResourceEvent.java} (80%) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java index f1e2fe5060..1a92a7a5f6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java @@ -86,7 +86,7 @@ private HashShuffleAppender getAppender(int partId) throws IOException { HashShuffleAppender appender = appenderMap.get(partId); if (appender == null) { appender = hashShuffleAppenderManager.getAppender(context.getConf(), - context.getQueryId().getTaskId().getExecutionBlockId(), partId, meta, outSchema); + context.getTaskId().getTaskId().getExecutionBlockId(), partId, meta, outSchema); appenderMap.put(partId, appender); } return appender; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java index de14c9a9d7..87a19a9f1d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java @@ -79,7 +79,7 @@ protected void fatal(Log log, String message) { } protected Path getExecutorTmpDir() { - return new Path(context.getQueryId().getTaskId().getExecutionBlockId().getQueryId().toString(), + return new Path(context.getTaskId().getTaskId().getExecutionBlockId().getQueryId().toString(), UUID.randomUUID().toString()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index af28886446..2a18de7ecb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -188,7 +188,9 @@ public void heartbeat( public void nodeHeartbeat(RpcController controller, TajoResourceTrackerProtocol.NodeHeartbeatRequestProto request, RpcCallback done) { //TODO implement with ResourceManager for scheduler - throw new RuntimeException(new ServiceException(new NotImplementedException().getMessage())); + TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.Builder + response = TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.newBuilder(); + done.run(response.setCommand(TajoResourceTrackerProtocol.ResponseCommand.NORMAL).build()); } private Worker createWorkerResource(NodeHeartbeat request) { diff --git a/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java b/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java new file mode 100644 index 0000000000..c424154ae8 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java @@ -0,0 +1,70 @@ +/** +* 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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.ShutdownHookManager; + +import java.lang.Thread.UncaughtExceptionHandler; + +/** + * This class is intended to be installed by calling + * {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)} + * In the main entry point. It is intended to try and cleanly shut down + * programs using the Yarn Event framework. + * + * Note: Right now it only will shut down the program if a Error is caught, but + * not any other exception. Anything else is just logged. + * + * this is an implementation copied from YarnUncaughtExceptionHandler + */ +public class TajoUncaughtExceptionHandler implements UncaughtExceptionHandler { + private static final Log LOG = LogFactory.getLog(TajoUncaughtExceptionHandler.class); + + @Override + public void uncaughtException(Thread t, Throwable e) { + if(ShutdownHookManager.get().isShutdownInProgress()) { + LOG.error("Thread " + t + " threw an Throwable, but we are shutting " + + "down, so ignoring this", e); + } else if(e instanceof Error) { + try { + LOG.fatal("Thread " + t + " threw an Error.", e); + } catch (Throwable err) { + //We don't want to not exit because of an issue with logging + } + + if(e instanceof OutOfMemoryError) { + //After catching an OOM java says it is undefined behavior, so don't + //even try to clean up or we can get stuck on shutdown. + try { + System.err.println("Halting due to Out Of Memory Error..."); + } catch (Throwable err) { + //Again we done want to exit because of logging issues. + } + ExitUtil.halt(-1); + } else { + //ExitUtil.terminate(-1); + } + } else { + LOG.error("Thread " + t + " threw an Exception.", e); + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java index e8ba3046b9..daced3e90c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java @@ -217,7 +217,7 @@ class WriterThread extends Thread { public void run() { LOG.info("HistoryWriter_" + processName + " started."); SimpleDateFormat df = new SimpleDateFormat("yyyyMMddHH"); - while (!stopped.get()) { + while (!stopped.get() && !Thread.interrupted()) { List> histories = Lists.newArrayList(); try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 0cc330468f..9e4a60f758 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -30,6 +30,7 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.TaskId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.QueryMasterProtocol; @@ -45,7 +46,6 @@ import java.net.ConnectException; import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; @@ -61,10 +61,10 @@ public class ExecutionBlockContext { private static final Log LOG = LogFactory.getLog(ExecutionBlockContext.class); private TaskRunnerManager manager; - public AtomicInteger completedTasksNum = new AtomicInteger(); - public AtomicInteger succeededTasksNum = new AtomicInteger(); - public AtomicInteger killedTasksNum = new AtomicInteger(); - public AtomicInteger failedTasksNum = new AtomicInteger(); + protected AtomicInteger completedTasksNum = new AtomicInteger(); + protected AtomicInteger succeededTasksNum = new AtomicInteger(); + protected AtomicInteger killedTasksNum = new AtomicInteger(); + protected AtomicInteger failedTasksNum = new AtomicInteger(); private FileSystem localFS; // for input files @@ -95,17 +95,18 @@ public class ExecutionBlockContext { // It keeps all of the query unit attempts while a TaskRunner is running. private final ConcurrentMap tasks = Maps.newConcurrentMap(); + @Deprecated private final ConcurrentMap histories = Maps.newConcurrentMap(); - public ExecutionBlockContext(TajoConf conf, TajoWorker.WorkerContext workerContext, - TaskRunnerManager manager, QueryContext queryContext, String plan, - ExecutionBlockId executionBlockId, WorkerConnectionInfo queryMaster, - PlanProto.ShuffleType shuffleType) throws Throwable { + private final Map taskHistories = Maps.newTreeMap(); + + public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, + TaskRunnerManager manager, RunExecutionBlockRequestProto request) throws IOException { this.manager = manager; - this.executionBlockId = executionBlockId; + this.executionBlockId = new ExecutionBlockId(request.getExecutionBlockId()); this.connManager = RpcClientManager.getInstance(); - this.queryMaster = queryMaster; - this.systemConf = conf; + this.queryMaster = new WorkerConnectionInfo(request.getQueryMaster()); + this.systemConf = workerContext.getConf(); this.reporter = new Reporter(); this.defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf); this.localFS = FileSystem.getLocal(systemConf); @@ -113,11 +114,11 @@ public ExecutionBlockContext(TajoConf conf, TajoWorker.WorkerContext workerConte // Setup QueryEngine according to the query plan // Here, we can setup row-based query engine or columnar query engine. this.queryEngine = new TajoQueryEngine(systemConf); - this.queryContext = queryContext; - this.plan = plan; + this.queryContext = new QueryContext(workerContext.getConf(), request.getQueryContext()); + this.plan = request.getPlanJson(); this.resource = new ExecutionBlockSharedResource(); this.workerContext = workerContext; - this.shuffleType = shuffleType; + this.shuffleType = request.getShuffleType(); } public void init() throws Throwable { @@ -131,7 +132,8 @@ public void init() throws Throwable { UserGroupInformation.setConfiguration(systemConf); // TODO - 'load credential' should be implemented // Getting taskOwner - UserGroupInformation taskOwner = UserGroupInformation.createRemoteUser(systemConf.getVar(TajoConf.ConfVars.USERNAME)); + UserGroupInformation + taskOwner = UserGroupInformation.createRemoteUser(systemConf.getVar(TajoConf.ConfVars.USERNAME)); // initialize DFS and LocalFileSystems this.taskOwner = taskOwner; @@ -144,7 +146,7 @@ public void init() throws Throwable { try { getStub().killQuery(null, executionBlockId.getQueryId().getProto(), NullCallback.get()); } catch (Throwable t) { - //ignore + LOG.error(t); } throw e; } @@ -183,9 +185,9 @@ public void stop(){ // If ExecutionBlock is stopped, all running or pending tasks will be marked as failed. for (Task task : tasks.values()) { - if (task.getStatus() == TajoProtos.TaskAttemptState.TA_PENDING || - task.getStatus() == TajoProtos.TaskAttemptState.TA_RUNNING) { - task.setState(TajoProtos.TaskAttemptState.TA_FAILED); + if (task.getTaskContext().getState() == TajoProtos.TaskAttemptState.TA_PENDING || + task.getTaskContext().getState() == TajoProtos.TaskAttemptState.TA_RUNNING) { + try{ task.abort(); } catch (Throwable e){ @@ -194,7 +196,7 @@ public void stop(){ } } tasks.clear(); - + taskHistories.clear(); resource.release(); RpcClientManager.cleanup(client); } @@ -253,18 +255,40 @@ public Task getTask(TaskAttemptId taskAttemptId){ return tasks.get(taskAttemptId); } + @Deprecated public void stopTaskRunner(String id){ manager.stopTaskRunner(id); } + @Deprecated public TaskRunner getTaskRunner(String taskRunnerId){ return manager.getTaskRunner(taskRunnerId); } + @Deprecated public void addTaskHistory(String taskRunnerId, TaskAttemptId quAttemptId, TaskHistory taskHistory) { histories.get(taskRunnerId).addTaskHistory(quAttemptId, taskHistory); } + public void addTaskHistory(TaskId taskId, TaskHistory taskHistory) { + taskHistories.put(taskId, taskHistory); + } + + public Map getTaskHistories() { + return taskHistories; + } + + public void fatalError(TaskAttemptId taskAttemptId, String message) { + if (message == null) { + message = "No error message"; + } + TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder() + .setId(taskAttemptId.getProto()) + .setErrorMessage(message); + + getStub().fatalError(null, builder.build(), NullCallback.get()); + } + public TaskRunnerHistory createTaskRunnerHistory(TaskRunner runner){ histories.putIfAbsent(runner.getId(), new TaskRunnerHistory(runner.getContainerId(), executionBlockId)); return histories.get(runner.getId()); @@ -355,7 +379,6 @@ private void sendHashShuffleReport(ExecutionBlockId ebId) throws Exception { protected class Reporter { private Thread reporterThread; - private AtomicBoolean reporterStop = new AtomicBoolean(); private static final int PROGRESS_INTERVAL = 1000; private static final int MAX_RETRIES = 10; @@ -374,7 +397,7 @@ Runnable createReporterThread() { int remainingRetries = MAX_RETRIES; @Override public void run() { - while (!reporterStop.get() && !Thread.interrupted()) { + while (!isStopped() && !Thread.interrupted()) { try { Interface masterStub = getStub(); @@ -384,13 +407,11 @@ public void run() { } else { for (Task task : new ArrayList(tasks.values())){ - if (task.isRunning() && task.isProgressChanged()) { - task.updateProgress(); + if (task.getTaskContext().getState() == + TajoProtos.TaskAttemptState.TA_RUNNING && task.isProgressChanged()) { masterStub.statusUpdate(null, task.getReport(), NullCallback.get()); - task.getContext().setProgressChanged(false); - } else { - task.updateProgress(); } + task.updateProgress(); } } } catch (Throwable t) { @@ -402,7 +423,7 @@ public void run() { throw new RuntimeException(t); } } finally { - if (remainingRetries > 0 && !reporterStop.get()) { + if (remainingRetries > 0 && !isStopped()) { synchronized (reporterThread) { try { reporterThread.wait(PROGRESS_INTERVAL); @@ -417,10 +438,6 @@ public void run() { } public void stop() throws InterruptedException { - if (reporterStop.getAndSet(true)) { - return; - } - if (reporterThread != null) { // Intent of the lock is to not send an interupt in the middle of an // umbilical.ping or umbilical.statusUpdate diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java new file mode 100644 index 0000000000..0721ef162e --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java @@ -0,0 +1,844 @@ +/** + * 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.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.netty.handler.codec.http.QueryStringDecoder; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.TaskAttemptState; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.planner.physical.PhysicalExec; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.ipc.QueryMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.*; +import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.TajoScriptEngine; +import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.serder.LogicalNodeDeserializer; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.rpc.NullCallback; +import org.apache.tajo.storage.*; +import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.util.NetUtils; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.URI; +import java.util.*; +import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; + +import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; + +@Deprecated +public class LegacyTaskImpl implements Task { + private static final Log LOG = LogFactory.getLog(LegacyTaskImpl.class); + private static final float FETCHER_PROGRESS = 0.5f; + + private final TajoConf systemConf; + private final QueryContext queryContext; + private final ExecutionBlockContext executionBlockContext; + private final String taskRunnerId; + + private final Path taskDir; + private final TaskRequest request; + private TaskAttemptContext context; + private List fetcherRunners; + private LogicalNode plan; + private final Map descs = Maps.newHashMap(); + private PhysicalExec executor; + private boolean interQuery; + private Path inputTableBaseDir; + + private long startTime; + private long finishTime; + + private final TableStats inputStats; + private List localChunks; + + // TODO - to be refactored + private ShuffleType shuffleType = null; + private Schema finalSchema = null; + private TupleComparator sortComp = null; + + public LegacyTaskImpl(String taskRunnerId, + Path baseDir, + TaskAttemptId taskId, + final ExecutionBlockContext executionBlockContext, + final TaskRequest request) throws IOException { + this(taskRunnerId, baseDir, taskId, executionBlockContext.getConf(), executionBlockContext, request); + } + + public LegacyTaskImpl(String taskRunnerId, + Path baseDir, + TaskAttemptId taskId, + TajoConf conf, + final ExecutionBlockContext executionBlockContext, + final TaskRequest request) throws IOException { + this.taskRunnerId = taskRunnerId; + this.request = request; + + this.systemConf = conf; + this.queryContext = request.getQueryContext(systemConf); + this.executionBlockContext = executionBlockContext; + this.taskDir = StorageUtil.concatPath(baseDir, + taskId.getTaskId().getId() + "_" + taskId.getId()); + + this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskId, + request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); + this.context.setDataChannel(request.getDataChannel()); + this.context.setEnforcer(request.getEnforcer()); + this.context.setState(TaskAttemptState.TA_PENDING); + this.inputStats = new TableStats(); + this.fetcherRunners = Lists.newArrayList(); + } + + public void initPlan() throws IOException { + plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); + LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); + if (scanNode != null) { + for (LogicalNode node : scanNode) { + ScanNode scan = (ScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); + if (partitionScanNode != null) { + for (LogicalNode node : partitionScanNode) { + PartitionedTableScanNode scan = (PartitionedTableScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + interQuery = request.getProto().getInterQuery(); + if (interQuery) { + context.setInterQuery(); + this.shuffleType = context.getDataChannel().getShuffleType(); + + if (shuffleType == ShuffleType.RANGE_SHUFFLE) { + SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); + this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); + this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); + } + } else { + Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) + .getAppenderFilePath(getId(), queryContext.getStagingDir()); + LOG.info("Output File Path: " + outFilePath); + context.setOutputPath(outFilePath); + } + + this.localChunks = Collections.synchronizedList(new ArrayList()); + LOG.info("=================================="); + LOG.info("* Stage " + request.getId() + " is initialized"); + LOG.info("* InterQuery: " + interQuery + + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + + ", Fragments (num: " + request.getFragments().size() + ")" + + ", Fetches (total:" + request.getFetches().size() + ") :"); + + if(LOG.isDebugEnabled()) { + for (FetchImpl f : request.getFetches()) { + LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); + } + } + LOG.info("* Local task dir: " + taskDir); + if(LOG.isDebugEnabled()) { + LOG.debug("* plan:\n"); + LOG.debug(plan.toString()); + } + LOG.info("=================================="); + } + + private void startScriptExecutors() throws IOException { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.start(systemConf); + } + } + + private void stopScriptExecutors() { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.shutdown(); + } + } + + @Override + public void init() throws IOException { + initPlan(); + startScriptExecutors(); + + if (context.getState() == TaskAttemptState.TA_PENDING) { + // initialize a task temporal dir + FileSystem localFS = executionBlockContext.getLocalFS(); + localFS.mkdirs(taskDir); + + if (request.getFetches().size() > 0) { + inputTableBaseDir = localFS.makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( + getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); + localFS.mkdirs(inputTableBaseDir); + Path tableDir; + for (String inputTable : context.getInputTables()) { + tableDir = new Path(inputTableBaseDir, inputTable); + if (!localFS.exists(tableDir)) { + LOG.info("the directory is created " + tableDir.toUri()); + localFS.mkdirs(tableDir); + } + } + } + // for localizing the intermediate data + fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); + } + } + + private TaskAttemptId getId() { + return context.getTaskId(); + } + + public String toString() { + return "queryId: " + this.getId() + " status: " + context.getState(); + } + + @Override + public boolean isStopped() { + return context.isStopped(); + } + + @Override + public TaskAttemptContext getTaskContext() { + return context; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return executionBlockContext; + } + + @Override + public boolean hasFetchPhase() { + return fetcherRunners.size() > 0; + } + + @Override + public void fetch() { + ExecutorService executorService = executionBlockContext.getTaskRunner(taskRunnerId).getFetchLauncher(); + for (Fetcher f : fetcherRunners) { + executorService.submit(new FetchRunner(context, f)); + } + } + + @Override + public void kill() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_KILLED); + context.stop(); + } + + @Override + public void abort() { + stopScriptExecutors(); + context.setState(TajoProtos.TaskAttemptState.TA_FAILED); + context.stop(); + } + + @Override + public TaskStatusProto getReport() { + TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); + builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); + builder.setId(context.getTaskId().getProto()) + .setProgress(context.getProgress()) + .setState(context.getState()); + + builder.setInputStats(reloadInputStats()); + + if (context.getResultStats() != null) { + builder.setResultStats(context.getResultStats().getProto()); + } + return builder.build(); + } + + @Override + public boolean isProgressChanged() { + return context.isProgressChanged(); + } + + @Override + public void updateProgress() { + if(context != null && context.isStopped()){ + return; + } + + if (executor != null && context.getProgress() < 1.0f) { + context.setExecutorProgress(executor.getProgress()); + } + } + + private CatalogProtos.TableStatsProto reloadInputStats() { + synchronized(inputStats) { + if (this.executor == null) { + return inputStats.getProto(); + } + + TableStats executorInputStats = this.executor.getInputStats(); + + if (executorInputStats != null) { + inputStats.setValues(executorInputStats); + } + return inputStats.getProto(); + } + } + + private TaskCompletionReport getTaskCompletionReport() { + TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); + builder.setId(context.getTaskId().getProto()); + + builder.setInputStats(reloadInputStats()); + + if (context.hasResultStats()) { + builder.setResultStats(context.getResultStats().getProto()); + } else { + builder.setResultStats(new TableStats().getProto()); + } + + Iterator> it = context.getShuffleFileOutputs(); + if (it.hasNext()) { + do { + Entry entry = it.next(); + ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); + part.setPartId(entry.getKey()); + + // Set output volume + if (context.getPartitionOutputVolume() != null) { + for (Entry e : context.getPartitionOutputVolume().entrySet()) { + if (entry.getKey().equals(e.getKey())) { + part.setVolume(e.getValue().longValue()); + break; + } + } + } + + builder.addShuffleFileOutputs(part.build()); + } while (it.hasNext()); + } + + return builder.build(); + } + + private void waitForFetch() throws InterruptedException, IOException { + context.getFetchLatch().await(); + LOG.info(context.getTaskId() + " All fetches are done!"); + Collection inputs = Lists.newArrayList(context.getInputTables()); + + // Get all broadcasted tables + Set broadcastTableNames = new HashSet(); + List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); + if (broadcasts != null) { + for (EnforceProperty eachBroadcast : broadcasts) { + broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); + } + } + + // localize the fetched data and skip the broadcast table + for (String inputTable: inputs) { + if (broadcastTableNames.contains(inputTable)) { + continue; + } + File tableDir = new File(context.getFetchIn(), inputTable); + FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); + context.updateAssignedFragments(inputTable, frags); + } + } + + @Override + public void run() throws Exception { + startTime = System.currentTimeMillis(); + Throwable error = null; + try { + if(!context.isStopped()) { + context.setState(TaskAttemptState.TA_RUNNING); + if (context.hasFetchPhase()) { + // If the fetch is still in progress, the query unit must wait for + // complete. + waitForFetch(); + context.setFetcherProgress(FETCHER_PROGRESS); + updateProgress(); + } + + this.executor = executionBlockContext.getTQueryEngine(). + createPlan(context, plan); + this.executor.init(); + + while(!context.isStopped() && executor.next() != null) { + } + } + } catch (Throwable e) { + error = e ; + LOG.error(e.getMessage(), e); + stopScriptExecutors(); + context.stop(); + } finally { + if (executor != null) { + try { + executor.close(); + reloadInputStats(); + } catch (IOException e) { + LOG.error(e, e); + } + this.executor = null; + } + + executionBlockContext.completedTasksNum.incrementAndGet(); + context.getHashShuffleAppenderManager().finalizeTask(getId()); + + QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); + if (context.isStopped()) { + context.setExecutorProgress(0.0f); + + if (context.getState() == TaskAttemptState.TA_KILLED) { + queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); + executionBlockContext.killedTasksNum.incrementAndGet(); + } else { + context.setState(TaskAttemptState.TA_FAILED); + TaskFatalErrorReport.Builder errorBuilder = + TaskFatalErrorReport.newBuilder() + .setId(getId().getProto()); + if (error != null) { + if (error.getMessage() == null) { + errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); + } else { + errorBuilder.setErrorMessage(error.getMessage()); + } + errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); + } + + queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); + executionBlockContext.failedTasksNum.incrementAndGet(); + } + } else { + // if successful + context.stop(); + context.setProgress(1.0f); + context.setState(TaskAttemptState.TA_SUCCEEDED); + executionBlockContext.succeededTasksNum.incrementAndGet(); + + TaskCompletionReport report = getTaskCompletionReport(); + queryMasterStub.done(null, report, NullCallback.get()); + } + finishTime = System.currentTimeMillis(); + LOG.info(context.getTaskId() + " completed. " + + "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + + ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() + + ", killed: " + executionBlockContext.killedTasksNum.intValue() + + ", failed: " + executionBlockContext.failedTasksNum.intValue()); + } + } + + @Override + public void cleanup() { + TaskHistory taskHistory = createTaskHistory(); + executionBlockContext.addTaskHistory(taskRunnerId, getId(), taskHistory); + executionBlockContext.getTasks().remove(getId()); + + fetcherRunners.clear(); + fetcherRunners = null; + try { + if(executor != null) { + executor.close(); + executor = null; + } + } catch (IOException e) { + LOG.fatal(e.getMessage(), e); + } + + executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); + stopScriptExecutors(); + } + + public TaskHistory createTaskHistory() { + TaskHistory taskHistory = null; + try { + taskHistory = new TaskHistory(context.getTaskId(), context.getState(), context.getProgress(), + startTime, finishTime, reloadInputStats()); + + if (context.getOutputPath() != null) { + taskHistory.setOutputPath(context.getOutputPath().toString()); + } + + if (context.getWorkDir() != null) { + taskHistory.setWorkingPath(context.getWorkDir().toString()); + } + + if (context.getResultStats() != null) { + taskHistory.setOutputStats(context.getResultStats().getProto()); + } + + if (hasFetchPhase()) { + taskHistory.setTotalFetchCount(fetcherRunners.size()); + int i = 0; + FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); + for (Fetcher fetcher : fetcherRunners) { + // TODO store the fetcher histories + if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { + builder.setStartTime(fetcher.getStartTime()); + builder.setFinishTime(fetcher.getFinishTime()); + builder.setFileLength(fetcher.getFileLen()); + builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); + builder.setState(fetcher.getState()); + + taskHistory.addFetcherHistory(builder.build()); + } + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; + } + taskHistory.setFinishedFetchCount(i); + } + } catch (Exception e) { + LOG.warn(e.getMessage(), e); + } + + return taskHistory; + } + + public int hashCode() { + return context.hashCode(); + } + + public boolean equals(Object obj) { + if (obj instanceof LegacyTaskImpl) { + LegacyTaskImpl other = (LegacyTaskImpl) obj; + return this.context.equals(other.context); + } + return false; + } + + private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) + throws IOException { + Configuration c = new Configuration(systemConf); + c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); + FileSystem fs = FileSystem.get(c); + Path tablePath = new Path(file.getAbsolutePath()); + + List listTablets = new ArrayList(); + FileFragment tablet; + + FileStatus[] fileLists = fs.listStatus(tablePath); + for (FileStatus f : fileLists) { + if (f.getLen() == 0) { + continue; + } + tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); + listTablets.add(tablet); + } + + // Special treatment for locally pseudo fetched chunks + synchronized (localChunks) { + for (FileChunk chunk : localChunks) { + if (name.equals(chunk.getEbId())) { + tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); + listTablets.add(tablet); + LOG.info("One local chunk is added to listTablets"); + } + } + } + + FileFragment[] tablets = new FileFragment[listTablets.size()]; + listTablets.toArray(tablets); + + return tablets; + } + + private class FetchRunner implements Runnable { + private final TaskAttemptContext ctx; + private final Fetcher fetcher; + private int maxRetryNum; + + public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { + this.ctx = ctx; + this.fetcher = fetcher; + this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); + } + + @Override + public void run() { + int retryNum = 0; + int retryWaitTime = 1000; //sec + + try { // for releasing fetch latch + while(!context.isStopped() && retryNum < maxRetryNum) { + if (retryNum > 0) { + try { + Thread.sleep(retryWaitTime); + retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds + } catch (InterruptedException e) { + LOG.error(e); + } + LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); + } + try { + FileChunk fetched = fetcher.get(); + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null + && fetched.getFile() != null) { + if (fetched.fromRemote() == false) { + localChunks.add(fetched); + LOG.info("Add a new FileChunk to local chunk list"); + } + break; + } + } catch (Throwable e) { + LOG.error("Fetch failed: " + fetcher.getURI(), e); + } + retryNum++; + } + } finally { + if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ + fetcherFinished(ctx); + } else { + if (retryNum == maxRetryNum) { + LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); + } + stopScriptExecutors(); + context.stop(); // retry task + ctx.getFetchLatch().countDown(); + } + } + } + } + + @VisibleForTesting + public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { + if (totalFetcher > 0) { + return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; + } else { + return 0.0f; + } + } + + private synchronized void fetcherFinished(TaskAttemptContext ctx) { + int fetcherSize = fetcherRunners.size(); + if(fetcherSize == 0) { + return; + } + + ctx.getFetchLatch().countDown(); + + int remainFetcher = (int) ctx.getFetchLatch().getCount(); + if (remainFetcher == 0) { + context.setFetcherProgress(FETCHER_PROGRESS); + } else { + context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); + } + } + + private List getFetchRunners(TaskAttemptContext ctx, + List fetches) throws IOException { + + if (fetches.size() > 0) { + Path inputDir = executionBlockContext.getLocalDirAllocator(). + getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); + + int i = 0; + File storeDir; + File defaultStoreFile; + FileChunk storeChunk = null; + List runnerList = Lists.newArrayList(); + + for (FetchImpl f : fetches) { + storeDir = new File(inputDir.toString(), f.getName()); + if (!storeDir.exists()) { + storeDir.mkdirs(); + } + + for (URI uri : f.getURIs()) { + defaultStoreFile = new File(storeDir, "in_" + i); + InetAddress address = InetAddress.getByName(uri.getHost()); + + WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); + if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { + boolean hasError = false; + try { + LOG.info("Try to get local file chunk at local host"); + storeChunk = getLocalStoredFileChunk(uri, systemConf); + } catch (Throwable t) { + hasError = true; + } + + // When a range request is out of range, storeChunk will be NULL. This case is normal state. + // So, we should skip and don't need to create storeChunk. + if (storeChunk == null && !hasError) { + continue; + } + + if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 + && hasError == false) { + storeChunk.setFromRemote(false); + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + + // If we decide that intermediate data should be really fetched from a remote host, storeChunk + // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it + storeChunk.setEbId(f.getName()); + Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); + LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); + runnerList.add(fetcher); + i++; + } + } + ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); + return runnerList; + } else { + return Lists.newArrayList(); + } + } + + private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { + // Parse the URI + LOG.info("getLocalStoredFileChunk starts"); + final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); + final List types = params.get("type"); + final List qids = params.get("qid"); + final List taskIdList = params.get("ta"); + final List stageIds = params.get("sid"); + final List partIds = params.get("p"); + final List offsetList = params.get("offset"); + final List lengthList = params.get("length"); + + if (types == null || stageIds == null || qids == null || partIds == null) { + LOG.error("Invalid URI - Required queryId, type, stage Id, and part id"); + return null; + } + + if (qids.size() != 1 && types.size() != 1 || stageIds.size() != 1) { + LOG.error("Invalid URI - Required qids, type, taskIds, stage Id, and part id"); + return null; + } + + String queryId = qids.get(0); + String shuffleType = types.get(0); + String sid = stageIds.get(0); + String partId = partIds.get(0); + + if (shuffleType.equals("r") && taskIdList == null) { + LOG.error("Invalid URI - For range shuffle, taskId is required"); + return null; + } + List taskIds = splitMaps(taskIdList); + + FileChunk chunk = null; + long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; + long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; + + LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId + + ", taskIds=" + taskIdList); + + // The working directory of Tajo worker for each query, including stage + String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; + + // If the stage requires a range shuffle + if (shuffleType.equals("r")) { + String ta = taskIds.get(0); + if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { + LOG.warn("Range shuffle - file not exist"); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); + String startKey = params.get("start").get(0); + String endKey = params.get("end").get(0); + boolean last = params.get("final") != null; + + try { + chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); + } catch (Throwable t) { + LOG.error("getFileChunks() throws exception"); + return null; + } + + // If the stage requires a hash shuffle or a scattered hash shuffle + } else if (shuffleType.equals("h") || shuffleType.equals("s")) { + int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf); + String partPath = queryBaseDir + "hash-shuffle/" + partParentId + "/" + partId; + if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { + LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); + File file = new File(path.toUri()); + long startPos = (offset >= 0 && length >= 0) ? offset : 0; + long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); + return null; + } + chunk = new FileChunk(file, startPos, readLen); + + } else { + LOG.error("Unknown shuffle type"); + return null; + } + + return chunk; + } + + private List splitMaps(List mapq) { + if (null == mapq) { + return null; + } + final List ret = new ArrayList(); + for (String s : mapq) { + Collections.addAll(ret, s.split(",")); + } + return ret; + } + + public static Path getTaskAttemptDir(TaskAttemptId quid) { + Path workDir = + StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), + String.valueOf(quid.getTaskId().getId()), + String.valueOf(quid.getId())); + return workDir; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index 20eec6b4e2..e763d13b4f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -30,27 +30,23 @@ import org.apache.tajo.storage.DiskUtil; import org.apache.tajo.unit.StorageUnit; import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.worker.event.NodeResourceAllocateEvent; -import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; -import org.apache.tajo.worker.event.NodeResourceManagerEvent; -import org.apache.tajo.worker.event.NodeStatusEvent; - -import java.util.concurrent.atomic.AtomicInteger; +import org.apache.tajo.worker.event.*; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; -public class NodeResourceManager extends AbstractService implements EventHandler { +public class NodeResourceManager extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(NodeResourceManager.class); private final Dispatcher dispatcher; + private final EventHandler taskEventHandler; private NodeResource totalResource; private NodeResource availableResource; - private AtomicInteger allocatedSize; private TajoConf tajoConf; - public NodeResourceManager(Dispatcher dispatcher){ + public NodeResourceManager(Dispatcher dispatcher, EventHandler taskEventHandler) { super(NodeResourceManager.class.getName()); this.dispatcher = dispatcher; + this.taskEventHandler = taskEventHandler; } @Override @@ -61,14 +57,14 @@ protected void serviceInit(Configuration conf) throws Exception { this.tajoConf = (TajoConf)conf; this.totalResource = createWorkerResource(tajoConf); this.availableResource = NodeResources.clone(totalResource); - this.dispatcher.register(NodeResourceManagerEvent.EventType.class, this); - this.allocatedSize = new AtomicInteger(); + this.dispatcher.register(NodeResourceEvent.EventType.class, this); + super.serviceInit(conf); LOG.info("Initialized NodeResourceManager for " + totalResource); } @Override - public void handle(NodeResourceManagerEvent event) { + public void handle(NodeResourceEvent event) { if (event instanceof NodeResourceAllocateEvent) { NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; @@ -76,22 +72,27 @@ public void handle(NodeResourceManagerEvent event) { for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { - allocatedSize.incrementAndGet(); - //TODO send task event to taskExecutor + if(allocateEvent.getRequest().hasExecutionBlockRequest()){ + //send ExecutionBlock start event to TaskManager + startExecutionBlock(allocateEvent.getRequest().getExecutionBlockRequest()); + } + + //send task start event to TaskExecutor + startTask(request.getTaskRequest(), resource); } else { + // reject the exceeded requests response.addCancellationTask(request); } } allocateEvent.getCallback().run(response.build()); } else if (event instanceof NodeResourceDeallocateEvent) { - allocatedSize.decrementAndGet(); NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; release(deallocateEvent.getResource()); // send current resource to ResourceTracker getDispatcher().getEventHandler().handle( - new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, getAvailableResource())); + new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); } } @@ -107,10 +108,6 @@ protected NodeResource getAvailableResource() { return availableResource; } - public int getAllocatedSize() { - return allocatedSize.get(); - } - private boolean allocate(NodeResource resource) { //TODO consider the jvm free memory if (NodeResources.fitsIn(resource, availableResource)) { @@ -120,6 +117,14 @@ private boolean allocate(NodeResource resource) { return false; } + protected void startExecutionBlock(RunExecutionBlockRequestProto request) { + taskEventHandler.handle(new ExecutionBlockStartEvent(request)); + } + + protected void startTask(TaskRequestProto request, NodeResource resource) { + taskEventHandler.handle(new TaskStartEvent(request, resource)); + } + private void release(NodeResource resource) { NodeResources.addTo(availableResource, resource); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 84ac419066..d13cd50a32 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -57,16 +57,16 @@ public class NodeStatusUpdater extends AbstractService implements EventHandler heartBeatRequestQueue; - private final WorkerConnectionInfo connectionInfo; + private final TajoWorker.WorkerContext workerContext; private final NodeResourceManager nodeResourceManager; private AsyncRpcClient rmClient; private ServiceTracker serviceTracker; private TajoResourceTrackerProtocolService.Interface resourceTracker; private int queueingLimit; - public NodeStatusUpdater(WorkerConnectionInfo connectionInfo, NodeResourceManager resourceManager) { + public NodeStatusUpdater(TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { super(NodeStatusUpdater.class.getSimpleName()); - this.connectionInfo = connectionInfo; + this.workerContext = workerContext; this.nodeResourceManager = resourceManager; } @@ -99,7 +99,8 @@ public void serviceStop() throws Exception { this.isStopped = true; synchronized (updaterThread) { - updaterThread.notifyAll(); + updaterThread.interrupt(); + updaterThread.join(); } super.serviceStop(); LOG.info("NodeStatusUpdater stopped."); @@ -107,14 +108,7 @@ public void serviceStop() throws Exception { @Override public void handle(NodeStatusEvent event) { - switch (event.getType()) { - case REPORT_RESOURCE: - heartBeatRequestQueue.add(event); //batch report to ResourceTracker - break; - case FLUSH_REPORTS: - heartBeatRequestQueue.add(event); //flush report to ResourceTracker - break; - } + heartBeatRequestQueue.add(event); } public int getQueueSize() { @@ -128,13 +122,13 @@ public int getQueueingLimit() { private NodeHeartbeatRequestProto createResourceReport(NodeResource resource) { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setAvailableResource(resource.getProto()); - requestProto.setWorkerId(connectionInfo.getId()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); return requestProto.build(); } private NodeHeartbeatRequestProto createHeartBeatReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setWorkerId(connectionInfo.getId()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); return requestProto.build(); } @@ -142,8 +136,8 @@ private NodeHeartbeatRequestProto createNodeStatusReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setTotalResource(nodeResourceManager.getTotalResource().getProto()); requestProto.setAvailableResource(nodeResourceManager.getAvailableResource().getProto()); - requestProto.setWorkerId(connectionInfo.getId()); - requestProto.setConnectionInfo(connectionInfo.getProto()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); + requestProto.setConnectionInfo(workerContext.getConnectionInfo().getProto()); //TODO set node status to requestProto.setStatus() return requestProto.build(); @@ -231,8 +225,8 @@ public void run() { } if (!events.isEmpty()) { - // send last available resource; - lastResponse = sendHeartbeat(createResourceReport(events.get(events.size() - 1).getResource())); + // send current available resource; + lastResponse = sendHeartbeat(createResourceReport(nodeResourceManager.getAvailableResource())); } else { // send ping; lastResponse = sendHeartbeat(createHeartBeatReport()); @@ -250,10 +244,10 @@ public void run() { } } catch (NoSuchMethodException nsme) { LOG.fatal(nsme.getMessage(), nsme); - Runtime.getRuntime().halt(1); + Runtime.getRuntime().halt(-1); } catch (ClassNotFoundException cnfe) { LOG.fatal(cnfe.getMessage(), cnfe); - Runtime.getRuntime().halt(1); + Runtime.getRuntime().halt(-1); } catch (Exception e) { LOG.error(e.getMessage(), e); if (!isStopped) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 4f07ca67d7..fbd070e5d0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -163,7 +163,7 @@ public void serviceInit(Configuration conf) throws Exception { serviceTracker = ServiceTrackerFactory.get(systemConf); - this.workerContext = new WorkerContext(); + this.workerContext = new TajoWorkerContext(); this.lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); String resourceManagerClassName = systemConf.getVar(ConfVars.RESOURCE_MANAGER_CLASS); @@ -386,7 +386,45 @@ public void serviceStop() throws Exception { LOG.info("TajoWorker main thread exiting"); } - public class WorkerContext { + public interface WorkerContext { + QueryMaster getQueryMaster(); + + TajoConf getConf(); + + ServiceTracker getServiceTracker(); + + QueryMasterManagerService getQueryMasterManagerService(); + + TaskRunnerManager getTaskRunnerManager(); + + CatalogService getCatalog(); + + WorkerConnectionInfo getConnectionInfo(); + + String getWorkerName(); + + LocalDirAllocator getLocalDirAllocator(); + + ClusterResourceSummary getClusterResource(); + + TajoSystemMetrics getWorkerSystemMetrics(); + + HashShuffleAppenderManager getHashShuffleAppenderManager(); + + HistoryWriter getTaskHistoryWriter(); + + HistoryReader getHistoryReader(); + + void cleanup(String strPath); + + void cleanupTemporalDirectories(); + + void setClusterResource(ClusterResourceSummary clusterResource); + + void setNumClusterNodes(int numClusterNodes); + } + + class TajoWorkerContext implements WorkerContext { public QueryMaster getQueryMaster() { if (queryMasterManagerService == null) { return null; @@ -430,7 +468,7 @@ public LocalDirAllocator getLocalDirAllocator(){ return lDirAllocator; } - protected void cleanup(String strPath) { + public void cleanup(String strPath) { if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); @@ -446,7 +484,7 @@ protected void cleanup(String strPath) { } } - protected void cleanupTemporalDirectories() { + public void cleanupTemporalDirectories() { if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); @@ -627,6 +665,7 @@ public static List getMountPath() throws IOException { } public static void main(String[] args) throws Exception { + Thread.setDefaultUncaughtExceptionHandler(new TajoUncaughtExceptionHandler()); StringUtils.startupShutdownMessage(TajoWorker.class, args, LOG); TajoConf tajoConf = new TajoConf(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index bbf8564566..de8afe8377 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -116,14 +116,7 @@ public void startExecutionBlock(RpcController controller, workerContext.getWorkerSystemMetrics().counter("query", "executedExecutionBlocksNum").inc(); try { - workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStartEvent( - new WorkerConnectionInfo(request.getQueryMaster()) - , new ExecutionBlockId(request.getExecutionBlockId()) - , request.getContainerId() - , new QueryContext(workerContext.getConf(), request.getQueryContext()), - request.getPlanJson(), - request.getShuffleType() - )); + workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStartEvent(request)); done.run(TajoWorker.TRUE_PROTO); } catch (Throwable t) { LOG.error(t.getMessage(), t); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index 4716dccad7..c84994003e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -18,839 +18,35 @@ package org.apache.tajo.worker; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import io.netty.handler.codec.http.QueryStringDecoder; -import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.tajo.TajoProtos; -import org.apache.tajo.TajoProtos.TaskAttemptState; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.catalog.TableDesc; -import org.apache.tajo.catalog.TableMeta; -import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.planner.physical.PhysicalExec; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.engine.query.TaskRequest; -import org.apache.tajo.ipc.QueryMasterProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.*; -import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.function.python.TajoScriptEngine; -import org.apache.tajo.plan.logical.*; -import org.apache.tajo.plan.serder.LogicalNodeDeserializer; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.pullserver.TajoPullServerService; -import org.apache.tajo.pullserver.retriever.FileChunk; -import org.apache.tajo.rpc.NullCallback; -import org.apache.tajo.storage.*; -import org.apache.tajo.storage.fragment.FileFragment; -import org.apache.tajo.util.NetUtils; +import org.apache.tajo.ipc.TajoWorkerProtocol; -import java.io.File; import java.io.IOException; -import java.net.InetAddress; -import java.net.URI; -import java.util.*; -import java.util.Map.Entry; -import java.util.concurrent.ExecutorService; -import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; -import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; +public interface Task { -public class Task { - private static final Log LOG = LogFactory.getLog(Task.class); - private static final float FETCHER_PROGRESS = 0.5f; + void init() throws IOException; - private final TajoConf systemConf; - private final QueryContext queryContext; - private final ExecutionBlockContext executionBlockContext; - private final TaskAttemptId taskId; - private final String taskRunnerId; + void fetch(); - private final Path taskDir; - private final TaskRequest request; - private TaskAttemptContext context; - private List fetcherRunners; - private LogicalNode plan; - private final Map descs = Maps.newHashMap(); - private PhysicalExec executor; - private boolean interQuery; - private Path inputTableBaseDir; + void run() throws Exception; - private long startTime; - private long finishTime; + void kill(); - private final TableStats inputStats; - private List localChunks; + void abort(); - // TODO - to be refactored - private ShuffleType shuffleType = null; - private Schema finalSchema = null; - private TupleComparator sortComp = null; + void cleanup(); - public Task(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { - this(taskRunnerId, baseDir, taskId, executionBlockContext.getConf(), executionBlockContext, request); - } + boolean hasFetchPhase(); - public Task(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - TajoConf conf, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { - this.taskRunnerId = taskRunnerId; - this.request = request; - this.taskId = taskId; + boolean isProgressChanged(); - this.systemConf = conf; - this.queryContext = request.getQueryContext(systemConf); - this.executionBlockContext = executionBlockContext; - this.taskDir = StorageUtil.concatPath(baseDir, - taskId.getTaskId().getId() + "_" + taskId.getId()); + boolean isStopped(); - this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskId, - request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); - this.context.setDataChannel(request.getDataChannel()); - this.context.setEnforcer(request.getEnforcer()); - this.context.setState(TaskAttemptState.TA_PENDING); - this.inputStats = new TableStats(); - this.fetcherRunners = Lists.newArrayList(); - } + void updateProgress(); - public void initPlan() throws IOException { - plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); - LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); - if (scanNode != null) { - for (LogicalNode node : scanNode) { - ScanNode scan = (ScanNode) node; - descs.put(scan.getCanonicalName(), scan.getTableDesc()); - } - } + TaskAttemptContext getTaskContext(); - LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); - if (partitionScanNode != null) { - for (LogicalNode node : partitionScanNode) { - PartitionedTableScanNode scan = (PartitionedTableScanNode) node; - descs.put(scan.getCanonicalName(), scan.getTableDesc()); - } - } + ExecutionBlockContext getExecutionBlockContext(); - interQuery = request.getProto().getInterQuery(); - if (interQuery) { - context.setInterQuery(); - this.shuffleType = context.getDataChannel().getShuffleType(); - - if (shuffleType == ShuffleType.RANGE_SHUFFLE) { - SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); - this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); - this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); - } - } else { - Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) - .getAppenderFilePath(taskId, queryContext.getStagingDir()); - LOG.info("Output File Path: " + outFilePath); - context.setOutputPath(outFilePath); - } - - this.localChunks = Collections.synchronizedList(new ArrayList()); - LOG.info("=================================="); - LOG.info("* Stage " + request.getId() + " is initialized"); - LOG.info("* InterQuery: " + interQuery - + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + - ", Fragments (num: " + request.getFragments().size() + ")" + - ", Fetches (total:" + request.getFetches().size() + ") :"); - - if(LOG.isDebugEnabled()) { - for (FetchImpl f : request.getFetches()) { - LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); - } - } - LOG.info("* Local task dir: " + taskDir); - if(LOG.isDebugEnabled()) { - LOG.debug("* plan:\n"); - LOG.debug(plan.toString()); - } - LOG.info("=================================="); - } - - private void startScriptExecutors() throws IOException { - for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.start(systemConf); - } - } - - private void stopScriptExecutors() { - for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.shutdown(); - } - } - - public void init() throws IOException { - initPlan(); - startScriptExecutors(); - - if (context.getState() == TaskAttemptState.TA_PENDING) { - // initialize a task temporal dir - FileSystem localFS = executionBlockContext.getLocalFS(); - localFS.mkdirs(taskDir); - - if (request.getFetches().size() > 0) { - inputTableBaseDir = localFS.makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( - getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); - localFS.mkdirs(inputTableBaseDir); - Path tableDir; - for (String inputTable : context.getInputTables()) { - tableDir = new Path(inputTableBaseDir, inputTable); - if (!localFS.exists(tableDir)) { - LOG.info("the directory is created " + tableDir.toUri()); - localFS.mkdirs(tableDir); - } - } - } - // for localizing the intermediate data - fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); - } - } - - public TaskAttemptId getTaskId() { - return taskId; - } - - public TaskAttemptId getId() { - return context.getTaskId(); - } - - public TaskAttemptState getStatus() { - return context.getState(); - } - - public String toString() { - return "queryId: " + this.getId() + " status: " + this.getStatus(); - } - - public void setState(TaskAttemptState status) { - context.setState(status); - } - - public TaskAttemptContext getContext() { - return context; - } - - public boolean hasFetchPhase() { - return fetcherRunners.size() > 0; - } - - public List getFetchers() { - return new ArrayList(fetcherRunners); - } - - public void fetch() { - ExecutorService executorService = executionBlockContext.getTaskRunner(taskRunnerId).getFetchLauncher(); - for (Fetcher f : fetcherRunners) { - executorService.submit(new FetchRunner(context, f)); - } - } - - public void kill() { - stopScriptExecutors(); - context.setState(TaskAttemptState.TA_KILLED); - context.stop(); - } - - public void abort() { - stopScriptExecutors(); - context.stop(); - } - - public void cleanUp() { - // remove itself from worker - if (context.getState() == TaskAttemptState.TA_SUCCEEDED) { - synchronized (executionBlockContext.getTasks()) { - executionBlockContext.getTasks().remove(this.getId()); - } - } else { - LOG.error("TaskAttemptId: " + context.getTaskId() + " status: " + context.getState()); - } - } - - public TaskStatusProto getReport() { - TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); - builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); - builder.setId(context.getTaskId().getProto()) - .setProgress(context.getProgress()) - .setState(context.getState()); - - builder.setInputStats(reloadInputStats()); - - if (context.getResultStats() != null) { - builder.setResultStats(context.getResultStats().getProto()); - } - return builder.build(); - } - - public boolean isRunning(){ - return context.getState() == TaskAttemptState.TA_RUNNING; - } - - public boolean isProgressChanged() { - return context.isProgressChanged(); - } - - public void updateProgress() { - if(context != null && context.isStopped()){ - return; - } - - if (executor != null && context.getProgress() < 1.0f) { - context.setExecutorProgress(executor.getProgress()); - } - } - - private CatalogProtos.TableStatsProto reloadInputStats() { - synchronized(inputStats) { - if (this.executor == null) { - return inputStats.getProto(); - } - - TableStats executorInputStats = this.executor.getInputStats(); - - if (executorInputStats != null) { - inputStats.setValues(executorInputStats); - } - return inputStats.getProto(); - } - } - - private TaskCompletionReport getTaskCompletionReport() { - TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); - builder.setId(context.getTaskId().getProto()); - - builder.setInputStats(reloadInputStats()); - - if (context.hasResultStats()) { - builder.setResultStats(context.getResultStats().getProto()); - } else { - builder.setResultStats(new TableStats().getProto()); - } - - Iterator> it = context.getShuffleFileOutputs(); - if (it.hasNext()) { - do { - Entry entry = it.next(); - ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); - part.setPartId(entry.getKey()); - - // Set output volume - if (context.getPartitionOutputVolume() != null) { - for (Entry e : context.getPartitionOutputVolume().entrySet()) { - if (entry.getKey().equals(e.getKey())) { - part.setVolume(e.getValue().longValue()); - break; - } - } - } - - builder.addShuffleFileOutputs(part.build()); - } while (it.hasNext()); - } - - return builder.build(); - } - - private void waitForFetch() throws InterruptedException, IOException { - context.getFetchLatch().await(); - LOG.info(context.getTaskId() + " All fetches are done!"); - Collection inputs = Lists.newArrayList(context.getInputTables()); - - // Get all broadcasted tables - Set broadcastTableNames = new HashSet(); - List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); - if (broadcasts != null) { - for (EnforceProperty eachBroadcast : broadcasts) { - broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); - } - } - - // localize the fetched data and skip the broadcast table - for (String inputTable: inputs) { - if (broadcastTableNames.contains(inputTable)) { - continue; - } - File tableDir = new File(context.getFetchIn(), inputTable); - FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); - context.updateAssignedFragments(inputTable, frags); - } - } - - public void run() throws Exception { - startTime = System.currentTimeMillis(); - Throwable error = null; - try { - if(!context.isStopped()) { - context.setState(TaskAttemptState.TA_RUNNING); - if (context.hasFetchPhase()) { - // If the fetch is still in progress, the query unit must wait for - // complete. - waitForFetch(); - context.setFetcherProgress(FETCHER_PROGRESS); - context.setProgressChanged(true); - updateProgress(); - } - - this.executor = executionBlockContext.getTQueryEngine(). - createPlan(context, plan); - this.executor.init(); - - while(!context.isStopped() && executor.next() != null) { - } - } - } catch (Throwable e) { - error = e ; - LOG.error(e.getMessage(), e); - stopScriptExecutors(); - context.stop(); - } finally { - if (executor != null) { - try { - executor.close(); - reloadInputStats(); - } catch (IOException e) { - LOG.error(e, e); - } - this.executor = null; - } - - executionBlockContext.completedTasksNum.incrementAndGet(); - context.getHashShuffleAppenderManager().finalizeTask(taskId); - - QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); - if (context.isStopped()) { - context.setExecutorProgress(0.0f); - - if (context.getState() == TaskAttemptState.TA_KILLED) { - queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); - executionBlockContext.killedTasksNum.incrementAndGet(); - } else { - context.setState(TaskAttemptState.TA_FAILED); - TaskFatalErrorReport.Builder errorBuilder = - TaskFatalErrorReport.newBuilder() - .setId(getId().getProto()); - if (error != null) { - if (error.getMessage() == null) { - errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); - } else { - errorBuilder.setErrorMessage(error.getMessage()); - } - errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); - } - - queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); - executionBlockContext.failedTasksNum.incrementAndGet(); - } - } else { - // if successful - context.setProgress(1.0f); - context.setState(TaskAttemptState.TA_SUCCEEDED); - executionBlockContext.succeededTasksNum.incrementAndGet(); - - TaskCompletionReport report = getTaskCompletionReport(); - queryMasterStub.done(null, report, NullCallback.get()); - } - finishTime = System.currentTimeMillis(); - LOG.info(context.getTaskId() + " completed. " + - "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + - ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() - + ", killed: " + executionBlockContext.killedTasksNum.intValue() - + ", failed: " + executionBlockContext.failedTasksNum.intValue()); - cleanupTask(); - } - } - - public void cleanupTask() { - TaskHistory taskHistory = createTaskHistory(); - executionBlockContext.addTaskHistory(taskRunnerId, getId(), taskHistory); - executionBlockContext.getTasks().remove(getId()); - - fetcherRunners.clear(); - fetcherRunners = null; - try { - if(executor != null) { - executor.close(); - executor = null; - } - } catch (IOException e) { - LOG.fatal(e.getMessage(), e); - } - - executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); - stopScriptExecutors(); - } - - public TaskHistory createTaskHistory() { - TaskHistory taskHistory = null; - try { - taskHistory = new TaskHistory(getTaskId(), getStatus(), context.getProgress(), - startTime, finishTime, reloadInputStats()); - - if (context.getOutputPath() != null) { - taskHistory.setOutputPath(context.getOutputPath().toString()); - } - - if (context.getWorkDir() != null) { - taskHistory.setWorkingPath(context.getWorkDir().toString()); - } - - if (context.getResultStats() != null) { - taskHistory.setOutputStats(context.getResultStats().getProto()); - } - - if (hasFetchPhase()) { - taskHistory.setTotalFetchCount(fetcherRunners.size()); - int i = 0; - FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); - for (Fetcher fetcher : fetcherRunners) { - // TODO store the fetcher histories - if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { - builder.setStartTime(fetcher.getStartTime()); - builder.setFinishTime(fetcher.getFinishTime()); - builder.setFileLength(fetcher.getFileLen()); - builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); - builder.setState(fetcher.getState()); - - taskHistory.addFetcherHistory(builder.build()); - } - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; - } - taskHistory.setFinishedFetchCount(i); - } - } catch (Exception e) { - LOG.warn(e.getMessage(), e); - } - - return taskHistory; - } - - public int hashCode() { - return context.hashCode(); - } - - public boolean equals(Object obj) { - if (obj instanceof Task) { - Task other = (Task) obj; - return this.context.equals(other.context); - } - return false; - } - - private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) - throws IOException { - Configuration c = new Configuration(systemConf); - c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); - FileSystem fs = FileSystem.get(c); - Path tablePath = new Path(file.getAbsolutePath()); - - List listTablets = new ArrayList(); - FileFragment tablet; - - FileStatus[] fileLists = fs.listStatus(tablePath); - for (FileStatus f : fileLists) { - if (f.getLen() == 0) { - continue; - } - tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); - listTablets.add(tablet); - } - - // Special treatment for locally pseudo fetched chunks - synchronized (localChunks) { - for (FileChunk chunk : localChunks) { - if (name.equals(chunk.getEbId())) { - tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); - listTablets.add(tablet); - LOG.info("One local chunk is added to listTablets"); - } - } - } - - FileFragment[] tablets = new FileFragment[listTablets.size()]; - listTablets.toArray(tablets); - - return tablets; - } - - private class FetchRunner implements Runnable { - private final TaskAttemptContext ctx; - private final Fetcher fetcher; - private int maxRetryNum; - - public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { - this.ctx = ctx; - this.fetcher = fetcher; - this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); - } - - @Override - public void run() { - int retryNum = 0; - int retryWaitTime = 1000; //sec - - try { // for releasing fetch latch - while(!context.isStopped() && retryNum < maxRetryNum) { - if (retryNum > 0) { - try { - Thread.sleep(retryWaitTime); - retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds - } catch (InterruptedException e) { - LOG.error(e); - } - LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); - } - try { - FileChunk fetched = fetcher.get(); - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null - && fetched.getFile() != null) { - if (fetched.fromRemote() == false) { - localChunks.add(fetched); - LOG.info("Add a new FileChunk to local chunk list"); - } - break; - } - } catch (Throwable e) { - LOG.error("Fetch failed: " + fetcher.getURI(), e); - } - retryNum++; - } - } finally { - if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ - fetcherFinished(ctx); - } else { - if (retryNum == maxRetryNum) { - LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); - } - stopScriptExecutors(); - context.stop(); // retry task - ctx.getFetchLatch().countDown(); - } - } - } - } - - @VisibleForTesting - public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { - if (totalFetcher > 0) { - return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; - } else { - return 0.0f; - } - } - - private synchronized void fetcherFinished(TaskAttemptContext ctx) { - int fetcherSize = fetcherRunners.size(); - if(fetcherSize == 0) { - return; - } - - ctx.getFetchLatch().countDown(); - - int remainFetcher = (int) ctx.getFetchLatch().getCount(); - if (remainFetcher == 0) { - context.setFetcherProgress(FETCHER_PROGRESS); - } else { - context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); - context.setProgressChanged(true); - } - } - - private List getFetchRunners(TaskAttemptContext ctx, - List fetches) throws IOException { - - if (fetches.size() > 0) { - Path inputDir = executionBlockContext.getLocalDirAllocator(). - getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); - - int i = 0; - File storeDir; - File defaultStoreFile; - FileChunk storeChunk = null; - List runnerList = Lists.newArrayList(); - - for (FetchImpl f : fetches) { - storeDir = new File(inputDir.toString(), f.getName()); - if (!storeDir.exists()) { - storeDir.mkdirs(); - } - - for (URI uri : f.getURIs()) { - defaultStoreFile = new File(storeDir, "in_" + i); - InetAddress address = InetAddress.getByName(uri.getHost()); - - WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); - if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { - boolean hasError = false; - try { - LOG.info("Try to get local file chunk at local host"); - storeChunk = getLocalStoredFileChunk(uri, systemConf); - } catch (Throwable t) { - hasError = true; - } - - // When a range request is out of range, storeChunk will be NULL. This case is normal state. - // So, we should skip and don't need to create storeChunk. - if (storeChunk == null && !hasError) { - continue; - } - - if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 - && hasError == false) { - storeChunk.setFromRemote(false); - } else { - storeChunk = new FileChunk(defaultStoreFile, 0, -1); - storeChunk.setFromRemote(true); - } - } else { - storeChunk = new FileChunk(defaultStoreFile, 0, -1); - storeChunk.setFromRemote(true); - } - - // If we decide that intermediate data should be really fetched from a remote host, storeChunk - // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it - storeChunk.setEbId(f.getName()); - Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); - LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); - runnerList.add(fetcher); - i++; - } - } - ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); - return runnerList; - } else { - return Lists.newArrayList(); - } - } - - private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { - // Parse the URI - LOG.info("getLocalStoredFileChunk starts"); - final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); - final List types = params.get("type"); - final List qids = params.get("qid"); - final List taskIdList = params.get("ta"); - final List stageIds = params.get("sid"); - final List partIds = params.get("p"); - final List offsetList = params.get("offset"); - final List lengthList = params.get("length"); - - if (types == null || stageIds == null || qids == null || partIds == null) { - LOG.error("Invalid URI - Required queryId, type, stage Id, and part id"); - return null; - } - - if (qids.size() != 1 && types.size() != 1 || stageIds.size() != 1) { - LOG.error("Invalid URI - Required qids, type, taskIds, stage Id, and part id"); - return null; - } - - String queryId = qids.get(0); - String shuffleType = types.get(0); - String sid = stageIds.get(0); - String partId = partIds.get(0); - - if (shuffleType.equals("r") && taskIdList == null) { - LOG.error("Invalid URI - For range shuffle, taskId is required"); - return null; - } - List taskIds = splitMaps(taskIdList); - - FileChunk chunk = null; - long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; - long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; - - LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId - + ", taskIds=" + taskIdList); - - // The working directory of Tajo worker for each query, including stage - String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; - - // If the stage requires a range shuffle - if (shuffleType.equals("r")) { - String ta = taskIds.get(0); - if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { - LOG.warn("Range shuffle - file not exist"); - return null; - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); - String startKey = params.get("start").get(0); - String endKey = params.get("end").get(0); - boolean last = params.get("final") != null; - - try { - chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); - } catch (Throwable t) { - LOG.error("getFileChunks() throws exception"); - return null; - } - - // If the stage requires a hash shuffle or a scattered hash shuffle - } else if (shuffleType.equals("h") || shuffleType.equals("s")) { - int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf); - String partPath = queryBaseDir + "hash-shuffle/" + partParentId + "/" + partId; - if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { - LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); - return null; - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); - File file = new File(path.toUri()); - long startPos = (offset >= 0 && length >= 0) ? offset : 0; - long readLen = (offset >= 0 && length >= 0) ? length : file.length(); - - if (startPos >= file.length()) { - LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); - return null; - } - chunk = new FileChunk(file, startPos, readLen); - - } else { - LOG.error("Unknown shuffle type"); - return null; - } - - return chunk; - } - - private List splitMaps(List mapq) { - if (null == mapq) { - return null; - } - final List ret = new ArrayList(); - for (String s : mapq) { - Collections.addAll(ret, s.split(",")); - } - return ret; - } - - public static Path getTaskAttemptDir(TaskAttemptId quid) { - Path workDir = - StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), - String.valueOf(quid.getTaskId().getId()), - String.valueOf(quid.getId())); - return workDir; - } + TajoWorkerProtocol.TaskStatusProto getReport(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java index 58028ac014..762e087488 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java @@ -46,7 +46,6 @@ import java.util.*; import java.util.Map.Entry; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; @@ -60,13 +59,13 @@ public class TaskAttemptContext { private volatile TaskAttemptState state; private TableStats resultStats; - private TaskAttemptId queryId; + private TaskAttemptId taskId; private final Path workDir; private boolean needFetch = false; private CountDownLatch doneFetchPhaseSignal; private float progress = 0.0f; private float fetcherProgress = 0.0f; - private AtomicBoolean progressChanged = new AtomicBoolean(false); + private volatile boolean progressChanged; /** a map of shuffled file outputs */ private Map shuffleFileOutputs; @@ -87,7 +86,7 @@ public class TaskAttemptContext { private EvalContext evalContext = new EvalContext(); public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext executionBlockContext, - final TaskAttemptId queryId, + final TaskAttemptId taskId, final FragmentProto[] fragments, final Path workDir) { this.queryContext = queryContext; @@ -97,7 +96,7 @@ public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext this.sharedResource = executionBlockContext.getSharedResource(); } - this.queryId = queryId; + this.taskId = taskId; if (fragments != null) { for (FragmentProto t : fragments) { @@ -114,25 +113,15 @@ public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext this.workDir = workDir; this.shuffleFileOutputs = Maps.newHashMap(); - state = TaskAttemptState.TA_PENDING; + this.state = TaskAttemptState.TA_PENDING; this.partitionOutputVolume = Maps.newHashMap(); - - if (workerContext != null) { - this.hashShuffleAppenderManager = workerContext.getHashShuffleAppenderManager(); - } else { - try { - this.hashShuffleAppenderManager = new HashShuffleAppenderManager(queryContext.getConf()); - } catch (IOException e) { - LOG.error(e.getMessage(), e); - } - } } @VisibleForTesting - public TaskAttemptContext(final QueryContext queryContext, final TaskAttemptId queryId, + public TaskAttemptContext(final QueryContext queryContext, final TaskAttemptId taskAttemptId, final Fragment [] fragments, final Path workDir) { - this(queryContext, null, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); + this(queryContext, null, taskAttemptId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); } public TajoConf getConf() { @@ -310,7 +299,7 @@ public Path getWorkDir() { } public TaskAttemptId getTaskId() { - return this.queryId; + return this.taskId; } public float getProgress() { @@ -326,17 +315,11 @@ public void setProgress(float progress) { this.progress = progress; } - if (previousProgress != progress) { - setProgressChanged(true); - } + this.progressChanged = previousProgress != progress; } public boolean isProgressChanged() { - return progressChanged.get(); - } - - public void setProgressChanged(boolean changed){ - progressChanged.set(changed); + return progressChanged; } public void setExecutorProgress(float executorProgress) { @@ -355,7 +338,9 @@ public void setFetcherProgress(float fetcherProgress) { if(Float.isNaN(fetcherProgress) || Float.isInfinite(fetcherProgress)){ fetcherProgress = 0.0f; } + float previousProgress = this.fetcherProgress; this.fetcherProgress = fetcherProgress; + this.progressChanged = previousProgress != fetcherProgress; } public FragmentProto getTable(String id) { @@ -383,13 +368,13 @@ public Collection getInputTables() { } public int hashCode() { - return Objects.hashCode(queryId); + return Objects.hashCode(taskId); } public boolean equals(Object obj) { if (obj instanceof TaskAttemptContext) { TaskAttemptContext other = (TaskAttemptContext) obj; - return queryId.equals(other.getTaskId()); + return taskId.equals(other.getTaskId()); } else { return false; } @@ -399,11 +384,18 @@ public QueryContext getQueryContext() { return queryContext; } - public TaskAttemptId getQueryId() { - return queryId; - } - public HashShuffleAppenderManager getHashShuffleAppenderManager() { + if(hashShuffleAppenderManager == null) { + if (workerContext != null) { + this.hashShuffleAppenderManager = workerContext.getHashShuffleAppenderManager(); + } else { + try { + this.hashShuffleAppenderManager = new HashShuffleAppenderManager(queryContext.getConf()); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + } + } return hashShuffleAppenderManager; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java new file mode 100644 index 0000000000..2576726840 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java @@ -0,0 +1,85 @@ +/** + * 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.tajo.TajoProtos; + +/** + * The driver class for Tajo Task processing. + */ +public class TaskContainer implements Runnable { + private static final Log LOG = LogFactory.getLog(TaskContainer.class); + + private final TaskExecutor executor; + private final int sequenceId; + + public TaskContainer(int sequenceId, TaskExecutor executor) { + this.sequenceId = sequenceId; + this.executor = executor; + } + + @Override + public void run() { + while (!executor.isStopped()) { + + Task task = null; + try { + task = executor.getNextTask(); + + task.getExecutionBlockContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); + + if (LOG.isDebugEnabled()) { + LOG.debug(sequenceId + TaskContainer.class.getSimpleName() + + " got task:" + task.getTaskContext().getTaskId()); + } + + TaskAttemptContext taskAttemptContext = task.getTaskContext(); + if (taskAttemptContext.isStopped()) return; + + task.init(); + + if (task.hasFetchPhase()) { + task.fetch(); // The fetch is performed in an asynchronous way. + } + + if (!taskAttemptContext.isStopped()) { + task.run(); + } + + task.cleanup(); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + if (task != null) { + try { + task.abort(); + task.getExecutionBlockContext().fatalError(task.getTaskContext().getTaskId(), e.getMessage()); + } catch (Throwable t) { + LOG.fatal(t.getMessage(), t); + } + } + } finally { + if (task != null) { + executor.stopTask(task.getTaskContext().getTaskId()); + } + } + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java new file mode 100644 index 0000000000..299952e7ae --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.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.worker; + +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.TaskId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.engine.query.TaskRequestImpl; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.worker.event.*; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * TaskExecutor uses a number of threads equal to the number of slots available for running tasks on the Worker + */ +public class TaskExecutor extends AbstractService implements EventHandler { + private static final Log LOG = LogFactory.getLog(TaskExecutor.class); + + private final TaskManager taskManager; + private final EventHandler rmEventHandler; + private final Map allocatedResourceMap; + private final BlockingQueue taskQueue; + private final AtomicInteger runningTasks; + private ThreadPoolExecutor fetcherExecutor; + private ExecutorService threadPool; + private TajoConf tajoConf; + private volatile boolean isStopped; + + public TaskExecutor(TaskManager taskManager, EventHandler rmEventHandler) { + super(TaskExecutor.class.getName()); + this.taskManager = taskManager; + this.rmEventHandler = rmEventHandler; + this.allocatedResourceMap = Maps.newConcurrentMap(); + this.runningTasks = new AtomicInteger(); + this.taskQueue = new LinkedBlockingQueue(); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + + this.tajoConf = (TajoConf) conf; + this.taskManager.getDispatcher().register(TaskExecutorEvent.EventType.class, this); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + int nThreads = this.tajoConf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); + this.threadPool = Executors.newFixedThreadPool(nThreads, + new ThreadFactoryBuilder().setNameFormat("Task executor #%d").build()); + + //TODO move to tajoConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + int maxFetcherThreads = Runtime.getRuntime().availableProcessors() * 2; + this.fetcherExecutor = new ThreadPoolExecutor(Math.min(nThreads, maxFetcherThreads), + maxFetcherThreads, + 60L, TimeUnit.SECONDS, + new SynchronousQueue(true)); + + + for (int i = 0; i < nThreads; i++) { + threadPool.submit(new TaskContainer(i, this)); + } + + super.serviceStart(); + LOG.info("Started TaskExecutor[" + nThreads + "], Fetcher executor[" + maxFetcherThreads + "]"); + } + + @Override + protected void serviceStop() throws Exception { + isStopped = true; + + threadPool.shutdown(); + fetcherExecutor.shutdown(); + super.serviceStop(); + } + + public boolean isStopped() { + return isStopped; + } + + public int getRunningTasks() { + return runningTasks.get(); + } + + /** + * This will block until a task is available. + */ + protected Task getNextTask() { + Task task = null; + try { + task = taskQueue.take(); + } catch (InterruptedException e) { + LOG.fatal(e); + } + return task; + } + + @SuppressWarnings("unchecked") + protected void stopTask(TaskAttemptId taskId) { + runningTasks.decrementAndGet(); + rmEventHandler.handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId))); + } + + protected ExecutorService getFetcherExecutor() { + return fetcherExecutor; + } + + + protected Task createTask(ExecutionBlockContext executionBlockContext, + TajoWorkerProtocol.TaskRequestProto taskRequest) throws IOException { + Task task = null; + TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + if (executionBlockContext.getTasks().containsKey(taskAttemptId)) { + String errorMessage = "Duplicate Task Attempt: " + taskAttemptId; + LOG.error(errorMessage); + executionBlockContext.fatalError(taskAttemptId, errorMessage); + } else { + task = new TaskImpl(new TaskRequestImpl(taskRequest), executionBlockContext, getFetcherExecutor()); + executionBlockContext.getTasks().put(task.getTaskContext().getTaskId(), task); + } + return task; + } + + @Override + public void handle(TaskExecutorEvent event) { + + if (event instanceof TaskStartEvent) { + TaskStartEvent startEvent = (TaskStartEvent) event; + allocatedResourceMap.put(startEvent.getTaskId(), startEvent.getAllocatedResource()); + + ExecutionBlockContext context = taskManager.getExecutionBlockContext( + startEvent.getTaskId().getTaskId().getExecutionBlockId()); + + try { + Task task = createTask(context, startEvent.getTaskRequest()); + if (task != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Arrival task: " + task.getTaskContext().getTaskId() + + ", allocated resource: " + startEvent.getAllocatedResource()); + } + taskQueue.put(task); + runningTasks.incrementAndGet(); + context.getWorkerContext().getWorkerSystemMetrics() + .histogram("tasks", "running").update(runningTasks.get()); + } else { + LOG.warn("Release duplicate task resource: " + startEvent.getAllocatedResource()); + stopTask(startEvent.getTaskId()); + } + } catch (InterruptedException e) { + if (!isStopped) { + LOG.fatal(e.getMessage(), e); + } + } catch (IOException e) { + stopTask(startEvent.getTaskId()); + } + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java new file mode 100644 index 0000000000..be3960b0ca --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -0,0 +1,838 @@ +/** + * 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.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.netty.handler.codec.http.QueryStringDecoder; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.TaskAttemptState; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.planner.physical.PhysicalExec; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.ipc.QueryMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.*; +import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.TajoScriptEngine; +import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.serder.LogicalNodeDeserializer; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.rpc.NullCallback; +import org.apache.tajo.storage.*; +import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.util.NetUtils; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.URI; +import java.util.*; +import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; + +import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; + +public class TaskImpl implements Task { + private static final Log LOG = LogFactory.getLog(TaskImpl.class); + private static final float FETCHER_PROGRESS = 0.5f; + + private final TajoConf systemConf; + private final QueryContext queryContext; + private final ExecutionBlockContext executionBlockContext; + private final TaskRequest request; + private final Map descs; + private final TableStats inputStats; + private final ExecutorService fetcherExecutor; + private final Path taskDir; + + private final TaskAttemptContext context; + private List fetcherRunners; + private LogicalNode plan; + private PhysicalExec executor; + + private boolean interQuery; + private Path inputTableBaseDir; + + private long startTime; + private long endTime; + + private List localChunks; + // TODO - to be refactored + private ShuffleType shuffleType = null; + private Schema finalSchema = null; + + private TupleComparator sortComp = null; + + public TaskImpl(final TaskRequest request, + final ExecutionBlockContext executionBlockContext, + final ExecutorService fetcherExecutor) throws IOException { + + this.request = request; + this.executionBlockContext = executionBlockContext; + this.systemConf = executionBlockContext.getConf(); + this.queryContext = request.getQueryContext(systemConf); + this.inputStats = new TableStats(); + this.fetcherRunners = Lists.newArrayList(); + this.fetcherExecutor = fetcherExecutor; + this.descs = Maps.newHashMap(); + + Path baseDirPath = executionBlockContext.createBaseDir(); + LOG.info("Task basedir is created (" + baseDirPath +")"); + TaskAttemptId taskAttemptId = request.getId(); + + this.taskDir = StorageUtil.concatPath(baseDirPath, + taskAttemptId.getTaskId().getId() + "_" + taskAttemptId.getId()); + this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskAttemptId, + request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); + this.context.setDataChannel(request.getDataChannel()); + this.context.setEnforcer(request.getEnforcer()); + this.context.setState(TaskAttemptState.TA_PENDING); + } + + public void initPlan() throws IOException { + plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); + LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); + if (scanNode != null) { + for (LogicalNode node : scanNode) { + ScanNode scan = (ScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); + if (partitionScanNode != null) { + for (LogicalNode node : partitionScanNode) { + PartitionedTableScanNode scan = (PartitionedTableScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + interQuery = request.getProto().getInterQuery(); + if (interQuery) { + context.setInterQuery(); + this.shuffleType = context.getDataChannel().getShuffleType(); + + if (shuffleType == ShuffleType.RANGE_SHUFFLE) { + SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); + this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); + this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); + } + } else { + Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) + .getAppenderFilePath(getId(), queryContext.getStagingDir()); + LOG.info("Output File Path: " + outFilePath); + context.setOutputPath(outFilePath); + } + + this.localChunks = Collections.synchronizedList(new ArrayList()); + LOG.info("=================================="); + LOG.info("* Stage " + request.getId() + " is initialized"); + LOG.info("* InterQuery: " + interQuery + + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + + ", Fragments (num: " + request.getFragments().size() + ")" + + ", Fetches (total:" + request.getFetches().size() + ") :"); + + if(LOG.isDebugEnabled()) { + for (FetchImpl f : request.getFetches()) { + LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); + } + } + LOG.info("* Local task dir: " + taskDir); + if(LOG.isDebugEnabled()) { + LOG.debug("* plan:\n"); + LOG.debug(plan.toString()); + } + LOG.info("=================================="); + } + + private void startScriptExecutors() throws IOException { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.start(systemConf); + } + } + + private void stopScriptExecutors() { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.shutdown(); + } + } + + @Override + public void init() throws IOException { + LOG.info("Initializing: " + getId()); + + initPlan(); + startScriptExecutors(); + + if (context.getState() == TaskAttemptState.TA_PENDING) { + // initialize a task temporal dir + FileSystem localFS = executionBlockContext.getLocalFS(); + localFS.mkdirs(taskDir); + + if (request.getFetches().size() > 0) { + inputTableBaseDir = localFS.makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( + getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); + localFS.mkdirs(inputTableBaseDir); + Path tableDir; + for (String inputTable : context.getInputTables()) { + tableDir = new Path(inputTableBaseDir, inputTable); + if (!localFS.exists(tableDir)) { + LOG.info("the directory is created " + tableDir.toUri()); + localFS.mkdirs(tableDir); + } + } + } + // for localizing the intermediate data + fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); + } + } + + private TaskAttemptId getId() { + return context.getTaskId(); + } + + public String toString() { + return "TaskId: " + this.getId() + " Status: " + context.getState(); + } + + @Override + public boolean isStopped() { + return context.isStopped(); + } + + @Override + public TaskAttemptContext getTaskContext() { + return context; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return executionBlockContext; + } + + @Override + public boolean hasFetchPhase() { + return fetcherRunners.size() > 0; + } + + @Override + public void fetch() { + for (Fetcher f : fetcherRunners) { + fetcherExecutor.submit(new FetchRunner(context, f)); + } + } + + @Override + public void kill() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_KILLED); + context.stop(); + } + + @Override + public void abort() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_FAILED); + context.stop(); + } + + @Override + public TaskStatusProto getReport() { + TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); + builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); + builder.setId(context.getTaskId().getProto()) + .setProgress(context.getProgress()) + .setState(context.getState()); + + builder.setInputStats(reloadInputStats()); + + if (context.getResultStats() != null) { + builder.setResultStats(context.getResultStats().getProto()); + } + return builder.build(); + } + + @Override + public boolean isProgressChanged() { + return context.isProgressChanged(); + } + + @Override + public void updateProgress() { + if(context != null && context.isStopped()){ + return; + } + + if (executor != null && context.getProgress() < 1.0f) { + context.setExecutorProgress(executor.getProgress()); + } + } + + private CatalogProtos.TableStatsProto reloadInputStats() { + synchronized(inputStats) { + if (this.executor == null) { + return inputStats.getProto(); + } + + TableStats executorInputStats = this.executor.getInputStats(); + + if (executorInputStats != null) { + inputStats.setValues(executorInputStats); + } + return inputStats.getProto(); + } + } + + private TaskCompletionReport getTaskCompletionReport() { + TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); + builder.setId(context.getTaskId().getProto()); + + builder.setInputStats(reloadInputStats()); + + if (context.hasResultStats()) { + builder.setResultStats(context.getResultStats().getProto()); + } else { + builder.setResultStats(new TableStats().getProto()); + } + + Iterator> it = context.getShuffleFileOutputs(); + if (it.hasNext()) { + do { + Entry entry = it.next(); + ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); + part.setPartId(entry.getKey()); + + // Set output volume + if (context.getPartitionOutputVolume() != null) { + for (Entry e : context.getPartitionOutputVolume().entrySet()) { + if (entry.getKey().equals(e.getKey())) { + part.setVolume(e.getValue().longValue()); + break; + } + } + } + + builder.addShuffleFileOutputs(part.build()); + } while (it.hasNext()); + } + + return builder.build(); + } + + private void waitForFetch() throws InterruptedException, IOException { + context.getFetchLatch().await(); + LOG.info(context.getTaskId() + " All fetches are done!"); + Collection inputs = Lists.newArrayList(context.getInputTables()); + + // Get all broadcasted tables + Set broadcastTableNames = new HashSet(); + List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); + if (broadcasts != null) { + for (EnforceProperty eachBroadcast : broadcasts) { + broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); + } + } + + // localize the fetched data and skip the broadcast table + for (String inputTable: inputs) { + if (broadcastTableNames.contains(inputTable)) { + continue; + } + File tableDir = new File(context.getFetchIn(), inputTable); + FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); + context.updateAssignedFragments(inputTable, frags); + } + } + + @Override + public void run() throws Exception { + startTime = System.currentTimeMillis(); + Throwable error = null; + + try { + if(!context.isStopped()) { + context.setState(TajoProtos.TaskAttemptState.TA_RUNNING); + if (context.hasFetchPhase()) { + // If the fetch is still in progress, the query unit must wait for complete. + waitForFetch(); + context.setFetcherProgress(FETCHER_PROGRESS); + updateProgress(); + } + + this.executor = executionBlockContext.getTQueryEngine().createPlan(context, plan); + this.executor.init(); + + while(!context.isStopped() && executor.next() != null) { + } + } + } catch (Throwable e) { + error = e ; + LOG.error(e.getMessage(), e); + stopScriptExecutors(); + context.stop(); + } finally { + if (executor != null) { + try { + executor.close(); + reloadInputStats(); + } catch (IOException e) { + LOG.error(e, e); + } + this.executor = null; + } + + executionBlockContext.completedTasksNum.incrementAndGet(); + context.getHashShuffleAppenderManager().finalizeTask(getId()); + + QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); + if (context.isStopped()) { + context.setExecutorProgress(0.0f); + + if (context.getState() == TaskAttemptState.TA_KILLED) { + queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); + executionBlockContext.killedTasksNum.incrementAndGet(); + } else { + context.setState(TaskAttemptState.TA_FAILED); + TaskFatalErrorReport.Builder errorBuilder = + TaskFatalErrorReport.newBuilder() + .setId(getId().getProto()); + if (error != null) { + if (error.getMessage() == null) { + errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); + } else { + errorBuilder.setErrorMessage(error.getMessage()); + } + errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); + } + + queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); + executionBlockContext.failedTasksNum.incrementAndGet(); + } + } else { + // if successful + context.stop(); + context.setProgress(1.0f); + context.setState(TaskAttemptState.TA_SUCCEEDED); + executionBlockContext.succeededTasksNum.incrementAndGet(); + + TaskCompletionReport report = getTaskCompletionReport(); + queryMasterStub.done(null, report, NullCallback.get()); + } + endTime = System.currentTimeMillis(); + LOG.info(context.getTaskId() + " completed. " + + "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + + ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() + + ", killed: " + executionBlockContext.killedTasksNum.intValue() + + ", failed: " + executionBlockContext.failedTasksNum.intValue()); + } + } + + @Override + public void cleanup() { + TaskHistory taskHistory = createTaskHistory(); + executionBlockContext.addTaskHistory(getId().getTaskId(), taskHistory); + executionBlockContext.getTasks().remove(getId()); + + fetcherRunners.clear(); + fetcherRunners = null; + try { + if(executor != null) { + executor.close(); + executor = null; + } + } catch (IOException e) { + LOG.fatal(e.getMessage(), e); + } + + executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); + stopScriptExecutors(); + } + + public TaskHistory createTaskHistory() { + TaskHistory taskHistory = null; + try { + taskHistory = new TaskHistory(context.getTaskId(), context.getState(), context.getProgress(), + startTime, endTime, reloadInputStats()); + + if (context.getOutputPath() != null) { + taskHistory.setOutputPath(context.getOutputPath().toString()); + } + + if (context.getWorkDir() != null) { + taskHistory.setWorkingPath(context.getWorkDir().toString()); + } + + if (context.getResultStats() != null) { + taskHistory.setOutputStats(context.getResultStats().getProto()); + } + + if (hasFetchPhase()) { + taskHistory.setTotalFetchCount(fetcherRunners.size()); + int i = 0; + FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); + for (Fetcher fetcher : fetcherRunners) { + // TODO store the fetcher histories + if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { + builder.setStartTime(fetcher.getStartTime()); + builder.setFinishTime(fetcher.getFinishTime()); + builder.setFileLength(fetcher.getFileLen()); + builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); + builder.setState(fetcher.getState()); + + taskHistory.addFetcherHistory(builder.build()); + } + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; + } + taskHistory.setFinishedFetchCount(i); + } + } catch (Exception e) { + LOG.warn(e.getMessage(), e); + } + + return taskHistory; + } + + public int hashCode() { + return context.hashCode(); + } + + public boolean equals(Object obj) { + if (obj instanceof TaskImpl) { + TaskImpl other = (TaskImpl) obj; + return this.context.equals(other.context); + } + return false; + } + + private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) + throws IOException { + Configuration c = new Configuration(systemConf); + c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); + FileSystem fs = FileSystem.get(c); + Path tablePath = new Path(file.getAbsolutePath()); + + List listTablets = new ArrayList(); + FileFragment tablet; + + FileStatus[] fileLists = fs.listStatus(tablePath); + for (FileStatus f : fileLists) { + if (f.getLen() == 0) { + continue; + } + tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); + listTablets.add(tablet); + } + + // Special treatment for locally pseudo fetched chunks + synchronized (localChunks) { + for (FileChunk chunk : localChunks) { + if (name.equals(chunk.getEbId())) { + tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); + listTablets.add(tablet); + LOG.info("One local chunk is added to listTablets"); + } + } + } + + FileFragment[] tablets = new FileFragment[listTablets.size()]; + listTablets.toArray(tablets); + + return tablets; + } + + private class FetchRunner implements Runnable { + private final TaskAttemptContext ctx; + private final Fetcher fetcher; + private int maxRetryNum; + + public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { + this.ctx = ctx; + this.fetcher = fetcher; + this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); + } + + @Override + public void run() { + int retryNum = 0; + int retryWaitTime = 1000; //sec + + try { // for releasing fetch latch + while(!context.isStopped() && retryNum < maxRetryNum) { + if (retryNum > 0) { + try { + Thread.sleep(retryWaitTime); + retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds + } catch (InterruptedException e) { + LOG.error(e); + } + LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); + } + try { + FileChunk fetched = fetcher.get(); + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null + && fetched.getFile() != null) { + if (fetched.fromRemote() == false) { + localChunks.add(fetched); + LOG.info("Add a new FileChunk to local chunk list"); + } + break; + } + } catch (Throwable e) { + LOG.error("Fetch failed: " + fetcher.getURI(), e); + } + retryNum++; + } + } finally { + if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ + fetcherFinished(ctx); + } else { + if (retryNum == maxRetryNum) { + LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); + } + stopScriptExecutors(); + context.stop(); // retry task + ctx.getFetchLatch().countDown(); + } + } + } + } + + @VisibleForTesting + public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { + if (totalFetcher > 0) { + return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; + } else { + return 0.0f; + } + } + + private synchronized void fetcherFinished(TaskAttemptContext ctx) { + int fetcherSize = fetcherRunners.size(); + if(fetcherSize == 0) { + return; + } + + ctx.getFetchLatch().countDown(); + + int remainFetcher = (int) ctx.getFetchLatch().getCount(); + if (remainFetcher == 0) { + context.setFetcherProgress(FETCHER_PROGRESS); + } else { + context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); + } + } + + private List getFetchRunners(TaskAttemptContext ctx, + List fetches) throws IOException { + + if (fetches.size() > 0) { + Path inputDir = executionBlockContext.getLocalDirAllocator(). + getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); + + int i = 0; + File storeDir; + File defaultStoreFile; + FileChunk storeChunk = null; + List runnerList = Lists.newArrayList(); + + for (FetchImpl f : fetches) { + storeDir = new File(inputDir.toString(), f.getName()); + if (!storeDir.exists()) { + if (!storeDir.mkdirs()) throw new IOException("Failed to create " + storeDir); + } + + for (URI uri : f.getURIs()) { + defaultStoreFile = new File(storeDir, "in_" + i); + InetAddress address = InetAddress.getByName(uri.getHost()); + + WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); + if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { + boolean hasError = false; + try { + LOG.info("Try to get local file chunk at local host"); + storeChunk = getLocalStoredFileChunk(uri, systemConf); + } catch (Throwable t) { + hasError = true; + } + + // When a range request is out of range, storeChunk will be NULL. This case is normal state. + // So, we should skip and don't need to create storeChunk. + if (storeChunk == null && !hasError) { + continue; + } + + if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 + && hasError == false) { + storeChunk.setFromRemote(false); + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + + // If we decide that intermediate data should be really fetched from a remote host, storeChunk + // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it + storeChunk.setEbId(f.getName()); + Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); + LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); + runnerList.add(fetcher); + i++; + } + } + ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); + return runnerList; + } else { + return Lists.newArrayList(); + } + } + + private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { + // Parse the URI + LOG.info("getLocalStoredFileChunk starts"); + final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); + final List types = params.get("type"); + final List qids = params.get("qid"); + final List taskIdList = params.get("ta"); + final List stageIds = params.get("sid"); + final List partIds = params.get("p"); + final List offsetList = params.get("offset"); + final List lengthList = params.get("length"); + + if (types == null || stageIds == null || qids == null || partIds == null) { + LOG.error("Invalid URI - Required queryId, type, stage Id, and part id"); + return null; + } + + if (qids.size() != 1 && types.size() != 1 || stageIds.size() != 1) { + LOG.error("Invalid URI - Required qids, type, taskIds, stage Id, and part id"); + return null; + } + + String queryId = qids.get(0); + String shuffleType = types.get(0); + String sid = stageIds.get(0); + String partId = partIds.get(0); + + if (shuffleType.equals("r") && taskIdList == null) { + LOG.error("Invalid URI - For range shuffle, taskId is required"); + return null; + } + List taskIds = splitMaps(taskIdList); + + FileChunk chunk; + long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; + long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; + + LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId + + ", taskIds=" + taskIdList); + + // The working directory of Tajo worker for each query, including stage + String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; + + // If the stage requires a range shuffle + if (shuffleType.equals("r")) { + String ta = taskIds.get(0); + if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { + LOG.warn("Range shuffle - file not exist"); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); + String startKey = params.get("start").get(0); + String endKey = params.get("end").get(0); + boolean last = params.get("final") != null; + + try { + chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); + } catch (Throwable t) { + LOG.error("getFileChunks() throws exception"); + return null; + } + + // If the stage requires a hash shuffle or a scattered hash shuffle + } else if (shuffleType.equals("h") || shuffleType.equals("s")) { + int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf); + String partPath = queryBaseDir + "hash-shuffle/" + partParentId + "/" + partId; + if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { + LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); + File file = new File(path.toUri()); + long startPos = (offset >= 0 && length >= 0) ? offset : 0; + long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); + return null; + } + chunk = new FileChunk(file, startPos, readLen); + + } else { + LOG.error("Unknown shuffle type"); + return null; + } + + return chunk; + } + + private List splitMaps(List mapq) { + if (null == mapq) { + return null; + } + final List ret = new ArrayList(); + for (String s : mapq) { + Collections.addAll(ret, s.split(",")); + } + return ret; + } + + public static Path getTaskAttemptDir(TaskAttemptId quid) { + Path workDir = + StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), + String.valueOf(quid.getTaskId().getId()), + String.valueOf(quid.getId())); + return workDir; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java new file mode 100644 index 0000000000..7990a7213e --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -0,0 +1,180 @@ +/** + * 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.common.collect.Maps; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TajoIdProtos; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.TaskId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.worker.event.*; + +import java.io.IOException; +import java.util.*; + +/** + * A TaskManager is responsible for managing executionBlock resource and tasks. + * */ +public class TaskManager extends AbstractService implements EventHandler { + private static final Log LOG = LogFactory.getLog(TaskManager.class); + + private final TajoWorker.WorkerContext workerContext; + private final Map executionBlockContextMap; + private final Dispatcher dispatcher; + private final EventHandler rmEventHandler; + + private TajoConf tajoConf; + + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { + super(TaskManager.class.getName()); + + this.dispatcher = dispatcher; + this.workerContext = workerContext; + this.executionBlockContextMap = Maps.newHashMap(); + this.rmEventHandler = rmEventHandler; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + + this.tajoConf = (TajoConf)conf; + dispatcher.register(TaskManagerEvent.EventType.class, this); + super.serviceInit(conf); + } + + @Override + protected void serviceStop() throws Exception { + + for(ExecutionBlockContext context: executionBlockContextMap.values()) { + context.stop(); + } + executionBlockContextMap.clear(); + super.serviceStop(); + } + + protected Dispatcher getDispatcher() { + return dispatcher; + } + + protected TajoWorker.WorkerContext getWorkerContext() { + return workerContext; + } + + protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + try { + ExecutionBlockContext context = new ExecutionBlockContext(getWorkerContext(), null, request); + + context.init(); + return context; + } catch (Throwable e) { + LOG.fatal(e.getMessage(), e); + throw new RuntimeException(e); + } + } + + protected void stopExecutionBlock(ExecutionBlockContext context, + TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + + if(context != null){ + try { + context.getSharedResource().releaseBroadcastCache(context.getExecutionBlockId()); + context.sendShuffleReport(); + getWorkerContext().getTaskHistoryWriter().flushTaskHistories(); + } catch (Exception e) { + LOG.fatal(e.getMessage(), e); + throw new RuntimeException(e); + } finally { + context.stop(); + + /* cleanup intermediate files */ + for (TajoIdProtos.ExecutionBlockIdProto ebId : cleanupList.getExecutionBlockIdList()) { + String inputDir = ExecutionBlockContext.getBaseInputDir(new ExecutionBlockId(ebId)).toString(); + workerContext.cleanup(inputDir); + String outputDir = ExecutionBlockContext.getBaseOutputDir(new ExecutionBlockId(ebId)).toString(); + workerContext.cleanup(outputDir); + } + } + LOG.info("Stopped execution block:" + context.getExecutionBlockId()); + } + } + + @Override + public void handle(TaskManagerEvent event) { + LOG.info("======================== Processing " + event.getExecutionBlockId() + " of type " + event.getType()); + + if (event instanceof ExecutionBlockStartEvent) { + + //receive event from NodeResourceManager + if(!executionBlockContextMap.containsKey(event.getExecutionBlockId())) { + ExecutionBlockContext context = createExecutionBlock(((ExecutionBlockStartEvent) event).getRequestProto()); + executionBlockContextMap.put(context.getExecutionBlockId(), context); + } else { + LOG.warn("Already initialized ExecutionBlock: " + event.getExecutionBlockId()); + } + } else if (event instanceof ExecutionBlockStopEvent) { + //receive event from QueryMaster + rmEventHandler.handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + stopExecutionBlock(executionBlockContextMap.remove(event.getExecutionBlockId()), + ((ExecutionBlockStopEvent) event).getCleanupList()); + } + } + + protected ExecutionBlockContext getExecutionBlockContext(ExecutionBlockId executionBlockId) { + return executionBlockContextMap.get(executionBlockId); + } + + public Task getTaskByTaskAttemptId(TaskAttemptId taskAttemptId) { + ExecutionBlockContext context = executionBlockContextMap.get(taskAttemptId.getTaskId().getExecutionBlockId()); + if (context != null) { + return context.getTask(taskAttemptId); + } + return null; + } + + public List getTaskHistories(ExecutionBlockId executionblockId) throws IOException { + List histories = new ArrayList(); + ExecutionBlockContext context = executionBlockContextMap.get(executionblockId); + if (context != null) { + histories.addAll(context.getTaskHistories().values()); + } + //TODO get List from HistoryReader + return histories; + } + + public TaskHistory getTaskHistory(TaskId taskId) { + TaskHistory history = null; + ExecutionBlockContext context = executionBlockContextMap.get(taskId.getExecutionBlockId()); + if (context != null) { + history = context.getTaskHistories().get(taskId); + } + //TODO get TaskHistory from HistoryReader + return history; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java index 774f3585c3..207b47e9f8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java @@ -34,10 +34,8 @@ import org.apache.tajo.master.container.TajoContainerIdPBImpl; import org.apache.tajo.master.container.TajoConverterUtils; import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NettyClientBase; import org.apache.tajo.rpc.NullCallback; -import java.net.ConnectException; import java.util.concurrent.*; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; @@ -45,6 +43,7 @@ /** * The driver class for Tajo Task processing. */ +@Deprecated public class TaskRunner extends AbstractService { /** class logger */ private static final Log LOG = LogFactory.getLog(TaskRunner.class); @@ -256,7 +255,7 @@ public void run() { LOG.info("Initializing: " + taskAttemptId); Task task = null; try { - task = new Task(getId(), getTaskBaseDir(), taskAttemptId, executionBlockContext, + task = new LegacyTaskImpl(getId(), getTaskBaseDir(), taskAttemptId, executionBlockContext, new TaskRequestImpl(taskRequest)); getContext().getTasks().put(taskAttemptId, task); @@ -269,10 +268,11 @@ public void run() { } catch (Throwable t) { LOG.error(t.getMessage(), t); fatalError(qmClientService, taskAttemptId, t.getMessage()); + } finally { if(task != null) { - task.cleanupTask(); + task.cleanup(); } - } finally { + callFuture = null; taskRequest = null; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java index 5c97ba81d8..16d32d443d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java @@ -36,6 +36,7 @@ /** * The history class for TaskRunner processing. */ +@Deprecated public class TaskRunnerHistory implements ProtoObject { private Service.STATE state; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java index 734a8a5b4b..d18a26295e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java @@ -36,6 +36,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; +@Deprecated public class TaskRunnerManager extends CompositeService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class); @@ -154,14 +155,7 @@ public void handle(TaskRunnerEvent event) { if(context == null){ try { - context = new ExecutionBlockContext(getTajoConf(), - getWorkerContext(), - this, - startEvent.getQueryContext(), - startEvent.getPlan(), - startEvent.getExecutionBlockId(), - startEvent.getQueryMaster(), - startEvent.getShuffleType()); + context = new ExecutionBlockContext(getWorkerContext(), this, startEvent.getRequest()); context.init(); } catch (Throwable e) { LOG.fatal(e.getMessage(), e); @@ -170,7 +164,7 @@ public void handle(TaskRunnerEvent event) { executionBlockContextMap.put(event.getExecutionBlockId(), context); } - TaskRunner taskRunner = new TaskRunner(context, startEvent.getContainerId()); + TaskRunner taskRunner = new TaskRunner(context, startEvent.getRequest().getContainerId()); LOG.info("Start TaskRunner:" + taskRunner.getId()); taskRunnerMap.put(taskRunner.getId(), taskRunner); taskRunnerHistoryMap.put(taskRunner.getId(), taskRunner.getHistory()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java new file mode 100644 index 0000000000..85d74e2d5a --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java @@ -0,0 +1,35 @@ +/** + * 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.event; + +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.ipc.TajoWorkerProtocol; + +public class ExecutionBlockStartEvent extends TaskManagerEvent { + private TajoWorkerProtocol.RunExecutionBlockRequestProto requestProto; + + public ExecutionBlockStartEvent(TajoWorkerProtocol.RunExecutionBlockRequestProto requestProto) { + super(EventType.EB_START, new ExecutionBlockId(requestProto.getExecutionBlockId())); + this.requestProto = requestProto; + } + + public TajoWorkerProtocol.RunExecutionBlockRequestProto getRequestProto() { + return requestProto; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java new file mode 100644 index 0000000000..2b967ab1d8 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java @@ -0,0 +1,37 @@ +/** + * 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.event; + +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TajoIdProtos; +import org.apache.tajo.ipc.TajoWorkerProtocol; + +public class ExecutionBlockStopEvent extends TaskManagerEvent { + private TajoWorkerProtocol.ExecutionBlockListProto cleanupList; + + public ExecutionBlockStopEvent(TajoIdProtos.ExecutionBlockIdProto executionBlockId, + TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + super(EventType.EB_STOP, new ExecutionBlockId(executionBlockId)); + this.cleanupList = cleanupList; + } + + public TajoWorkerProtocol.ExecutionBlockListProto getCleanupList() { + return cleanupList; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java index 2f411e8c5c..9a1c106865 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -24,7 +24,7 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; -public class NodeResourceAllocateEvent extends NodeResourceManagerEvent { +public class NodeResourceAllocateEvent extends NodeResourceEvent { private BatchAllocationRequestProto request; private RpcCallback callback; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java index a298d77e20..31d9229ad4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java @@ -21,7 +21,7 @@ import org.apache.tajo.TajoProtos; import org.apache.tajo.resource.NodeResource; -public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { +public class NodeResourceDeallocateEvent extends NodeResourceEvent { private NodeResource resource; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java similarity index 80% rename from tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java rename to tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java index bcb34486c0..6fd2e0d517 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java @@ -19,16 +19,17 @@ package org.apache.tajo.worker.event; import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.resource.NodeResource; -public class NodeResourceManagerEvent extends AbstractEvent { +public class NodeResourceEvent extends AbstractEvent { + //consumer: NodeResourceManager public enum EventType { + // producer: TajoWorkerManagerService ALLOCATE, + // producer: TaskExecutor DEALLOCATE } - public NodeResourceManagerEvent(EventType eventType) { + public NodeResourceEvent(EventType eventType) { super(eventType); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java index 58ab74af0a..9eb8ae9d7b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java @@ -22,19 +22,16 @@ import org.apache.tajo.resource.NodeResource; public class NodeStatusEvent extends AbstractEvent { - private final NodeResource resource; + // consumer: NodeStatusUpdater public enum EventType { + // producer: NodeResourceManager REPORT_RESOURCE, + // producer: TaskManager FLUSH_REPORTS } - public NodeStatusEvent(EventType eventType, NodeResource resource) { + public NodeStatusEvent(EventType eventType) { super(eventType); - this.resource = resource; - } - - public NodeResource getResource() { - return resource; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java new file mode 100644 index 0000000000..c609c67150 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java @@ -0,0 +1,44 @@ +/** + * 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.event; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.tajo.TaskAttemptId; + +public class TaskExecutorEvent extends AbstractEvent { + + // producer: NodeResourceManager, consumer: TaskExecutorEvent + public enum EventType { + START, + KILL, + ABORT + } + + private TaskAttemptId taskAttemptId; + + public TaskExecutorEvent(EventType eventType, + TaskAttemptId taskAttemptId) { + super(eventType); + this.taskAttemptId = taskAttemptId; + } + + public TaskAttemptId getTaskId() { + return taskAttemptId; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java new file mode 100644 index 0000000000..39b541b097 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.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.worker.event; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TaskAttemptId; + +public class TaskManagerEvent extends AbstractEvent { + // producer: NodeResourceManager, consumer: TaskManager + public enum EventType { + EB_START, + EB_STOP + } + + private ExecutionBlockId executionBlockId; + + public TaskManagerEvent(EventType eventType, + ExecutionBlockId executionBlockId) { + super(eventType); + this.executionBlockId = executionBlockId; + } + + public ExecutionBlockId getExecutionBlockId() { + return executionBlockId; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java index aac8973b81..717525197d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java @@ -21,6 +21,7 @@ import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.ExecutionBlockId; +@Deprecated public class TaskRunnerEvent extends AbstractEvent { public enum EventType { START, diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java index 908afa2293..9406794254 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java @@ -20,48 +20,20 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; - +@Deprecated public class TaskRunnerStartEvent extends TaskRunnerEvent { - private final QueryContext queryContext; - private final WorkerConnectionInfo queryMaster; - private final String containerId; - private final String plan; - private final PlanProto.ShuffleType shuffleType; - - public TaskRunnerStartEvent(WorkerConnectionInfo queryMaster, - ExecutionBlockId executionBlockId, - String containerId, - QueryContext context, - String plan, - PlanProto.ShuffleType shuffleType) { - super(EventType.START, executionBlockId); - this.queryMaster = queryMaster; - this.containerId = containerId; - this.queryContext = context; - this.plan = plan; - this.shuffleType = shuffleType; - } - - public WorkerConnectionInfo getQueryMaster() { - return queryMaster; - } - - public String getContainerId() { - return containerId; - } - - public QueryContext getQueryContext() { - return queryContext; - } + private final TajoWorkerProtocol.RunExecutionBlockRequestProto request; - public String getPlan() { - return plan; + public TaskRunnerStartEvent(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + super(EventType.START, new ExecutionBlockId(request.getExecutionBlockId())); + this.request = request; } - public PlanProto.ShuffleType getShuffleType() { - return shuffleType; + public TajoWorkerProtocol.RunExecutionBlockRequestProto getRequest() { + return request; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java index c8ec20d138..297f30c037 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java @@ -20,6 +20,7 @@ import org.apache.tajo.ExecutionBlockId; +@Deprecated public class TaskRunnerStopEvent extends TaskRunnerEvent { public TaskRunnerStopEvent(ExecutionBlockId executionBlockId) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java new file mode 100644 index 0000000000..f60e7c4212 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java @@ -0,0 +1,44 @@ +/** + * 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.event; + +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.resource.NodeResource; +import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; + +public class TaskStartEvent extends TaskExecutorEvent { + + private NodeResource allocatedResource; + private TaskRequestProto taskRequest; + + public TaskStartEvent(TaskRequestProto taskRequest, + NodeResource allocatedResource) { + super(EventType.START, new TaskAttemptId(taskRequest.getId())); + this.taskRequest = taskRequest; + this.allocatedResource = allocatedResource; + } + + public NodeResource getAllocatedResource() { + return allocatedResource; + } + + public TaskRequestProto getTaskRequest() { + return taskRequest; + } +} diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 2324596b1b..715b1e681a 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -208,6 +208,7 @@ message TaskAllocationRequestProto { message BatchAllocationRequestProto { required ExecutionBlockIdProto executionBlockId = 1; repeated TaskAllocationRequestProto taskRequest = 2; + optional RunExecutionBlockRequestProto executionBlockRequest = 3; //TODO should be refactored } message BatchAllocationResponseProto { diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index eca7f6d7c4..0cec3dac49 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -19,6 +19,7 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Lists; +import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.tajo.*; @@ -33,16 +34,25 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequestImpl; -import org.apache.tajo.ipc.ClientProtos; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; import org.apache.tajo.plan.LogicalOptimizer; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.LogicalPlanner; import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.session.Session; +import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; import org.apache.tajo.worker.ExecutionBlockContext; -import org.apache.tajo.worker.Task; +import org.apache.tajo.worker.LegacyTaskImpl; +import org.apache.tajo.worker.TajoWorker; +import org.apache.tajo.worker.TaskRunnerManager; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -230,7 +240,7 @@ public void testKillTask() throws Throwable { QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId eid = QueryIdFactory.newExecutionBlockId(qid, 1); TaskId tid = QueryIdFactory.newTaskId(eid); - TajoConf conf = new TajoConf(); + final TajoConf conf = new TajoConf(); TaskRequestImpl taskRequest = new TaskRequestImpl(); taskRequest.set(null, new ArrayList(), @@ -238,18 +248,37 @@ public void testKillTask() throws Throwable { taskRequest.setInterQuery(); TaskAttemptId attemptId = new TaskAttemptId(tid, 1); + WorkerConnectionInfo queryMaster = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + requestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + + requestProto.setExecutionBlockId(eid.getProto()) + .setQueryMaster(queryMaster.getProto()) + .setNodeId(queryMaster.getHost()+":" + queryMaster.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + @Override + public TajoConf getConf() { + return conf; + } + }; + ExecutionBlockContext context = - new ExecutionBlockContext(conf, null, null, new QueryContext(conf), null, eid, null, null); + new ExecutionBlockContext(workerContext, null, requestProto.build()); - org.apache.tajo.worker.Task task = new Task("test", CommonTestingUtil.getTestDir(), attemptId, + org.apache.tajo.worker.Task task = new LegacyTaskImpl("test", CommonTestingUtil.getTestDir(), attemptId, conf, context, taskRequest); task.kill(); - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); try { task.run(); - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); } catch (Exception e) { - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); } } @@ -271,4 +300,94 @@ protected void dispatch(Event event) { super.dispatch(event); } } + + abstract class MockWorkerContext implements TajoWorker.WorkerContext { + + @Override + public QueryMaster getQueryMaster() { + return null; + } + + public abstract TajoConf getConf(); + + @Override + public ServiceTracker getServiceTracker() { + return null; + } + + @Override + public QueryMasterManagerService getQueryMasterManagerService() { + return null; + } + + @Override + public TaskRunnerManager getTaskRunnerManager() { + return null; + } + + @Override + public CatalogService getCatalog() { + return null; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + return null; + } + + @Override + public String getWorkerName() { + return null; + } + + @Override + public LocalDirAllocator getLocalDirAllocator() { + return null; + } + + @Override + public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { + return null; + } + + @Override + public TajoSystemMetrics getWorkerSystemMetrics() { + return null; + } + + @Override + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return null; + } + + @Override + public HistoryWriter getTaskHistoryWriter() { + return null; + } + + @Override + public HistoryReader getHistoryReader() { + return null; + } + + @Override + public void cleanup(String strPath) { + + } + + @Override + public void cleanupTemporalDirectories() { + + } + + @Override + public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { + + } + + @Override + public void setNumClusterNodes(int numClusterNodes) { + + } + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java new file mode 100644 index 0000000000..9d4e1f3392 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java @@ -0,0 +1,42 @@ +/** + * 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.tajo.TaskAttemptId; +import org.apache.tajo.ipc.TajoWorkerProtocol; + +import java.io.IOException; + +public class MockExecutionBlock extends ExecutionBlockContext { + + public MockExecutionBlock(TajoWorker.WorkerContext workerContext, + TajoWorkerProtocol.RunExecutionBlockRequestProto request) throws IOException { + super(workerContext, null, request); + } + + @Override + public void init() throws Throwable { + //skip + } + + @Override + public void fatalError(TaskAttemptId taskAttemptId, String message) { + + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java new file mode 100644 index 0000000000..18b9405c8a --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java @@ -0,0 +1,96 @@ +/** + * 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.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.QueryIdFactory; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.worker.event.NodeResourceEvent; + +import java.util.Queue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; + +public class MockNodeResourceManager extends NodeResourceManager { + + volatile boolean enableTaskHandlerEvent = true; + private final Semaphore barrier; + + public MockNodeResourceManager(Semaphore barrier, Dispatcher dispatcher, EventHandler taskEventHandler) { + super(dispatcher, taskEventHandler); + this.barrier = barrier; + } + + @Override + public void handle(NodeResourceEvent event) { + super.handle(event); + barrier.release(); + } + + @Override + protected void startExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + if(enableTaskHandlerEvent) { + super.startExecutionBlock(request); + } + } + + @Override + protected void startTask(TajoWorkerProtocol.TaskRequestProto request, NodeResource resource) { + if(enableTaskHandlerEvent) { + super.startTask(request, resource); + } + } + + /** + * skip task execution and deallocation for testing + * */ + public void setTaskHandlerEvent(boolean flag) { + enableTaskHandlerEvent = flag; + } + + protected static Queue createTaskRequests( + ExecutionBlockId ebId, int memory, int size) { + + Queue + requestProtoList = new LinkedBlockingQueue(); + for (int i = 0; i < size; i++) { + + TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(ebId, i), 0); + TajoWorkerProtocol.TaskRequestProto.Builder builder = + TajoWorkerProtocol.TaskRequestProto.newBuilder(); + builder.setId(taskAttemptId.getProto()); + builder.setShouldDie(true); + builder.setOutputTable(""); + builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); + builder.setClusteredOutput(false); + + + requestProtoList.add(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + .setResource(NodeResources.createResource(memory).getProto()) + .setTaskRequest(builder.build()).build()); + } + return requestProtoList; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java index 2d7d0be6d8..dfcfd4f435 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java @@ -39,9 +39,9 @@ public class MockNodeStatusUpdater extends NodeStatusUpdater { private Map resources = Maps.newHashMap(); private MockResourceTracker resourceTracker; - public MockNodeStatusUpdater(CountDownLatch barrier, WorkerConnectionInfo connectionInfo, + public MockNodeStatusUpdater(CountDownLatch barrier, TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { - super(connectionInfo, resourceManager); + super(workerContext, resourceManager); this.barrier = barrier; this.resourceTracker = new MockResourceTracker(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java new file mode 100644 index 0000000000..f62733f1a2 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -0,0 +1,141 @@ +/** + * 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.event.EventHandler; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.worker.event.TaskExecutorEvent; + +import java.io.IOException; +import java.util.concurrent.Semaphore; + +public class MockTaskExecutor extends TaskExecutor { + + protected final Semaphore barrier; + + public MockTaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { + super(taskManager, rmEventHandler); + this.barrier = barrier; + } + + @Override + public void handle(TaskExecutorEvent event) { + super.handle(event); + barrier.release(); + } + + @Override + protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + + //ignore status changed log + final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null) { + private TajoProtos.TaskAttemptState state; + + @Override + public TajoProtos.TaskAttemptState getState() { + return state; + } + + @Override + public void setState(TajoProtos.TaskAttemptState state) { + this.state = state; + } + }; + + return new Task() { + @Override + public void init() throws IOException { + + } + + @Override + public void fetch() { + + } + + @Override + public void run() throws Exception { + taskAttemptContext.stop(); + taskAttemptContext.setProgress(1.0f); + taskAttemptContext.setState(TajoProtos.TaskAttemptState.TA_SUCCEEDED); + } + + @Override + public void kill() { + + } + + @Override + public void abort() { + + } + + @Override + public void cleanup() { + + } + + @Override + public boolean hasFetchPhase() { + return false; + } + + @Override + public boolean isProgressChanged() { + return false; + } + + @Override + public boolean isStopped() { + return taskAttemptContext.isStopped(); + } + + @Override + public void updateProgress() { + + } + + @Override + public TaskAttemptContext getTaskContext() { + return taskAttemptContext; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return context; + } + + @Override + public TajoWorkerProtocol.TaskStatusProto getReport() { + TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + builder.setWorkerName("localhost:0"); + builder.setId(taskAttemptContext.getTaskId().getProto()) + .setProgress(taskAttemptContext.getProgress()) + .setState(taskAttemptContext.getState()); + + builder.setInputStats(new TableStats().getProto()); + return builder.build(); + } + }; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java new file mode 100644 index 0000000000..678b0636c4 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java @@ -0,0 +1,59 @@ +/** + * 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.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.worker.event.TaskManagerEvent; + +import java.io.IOException; +import java.util.concurrent.Semaphore; + +public class MockTaskManager extends TaskManager { + + private final Semaphore barrier; + + public MockTaskManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { + super(dispatcher, workerContext, rmEventHandler); + this.barrier = barrier; + } + + @Override + protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + try { + return new MockExecutionBlock(getWorkerContext(), request); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + protected void stopExecutionBlock(ExecutionBlockContext context, + TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + //skip for testing + } + + @Override + public void handle(TaskManagerEvent event) { + super.handle(event); + barrier.release(); + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java new file mode 100644 index 0000000000..e8c2b9c369 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java @@ -0,0 +1,129 @@ +/** + * 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.fs.LocalDirAllocator; +import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.querymaster.QueryMaster; +import org.apache.tajo.querymaster.QueryMasterManagerService; +import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; + +public abstract class MockWorkerContext implements TajoWorker.WorkerContext { + TajoSystemMetrics tajoSystemMetrics; + + @Override + public QueryMaster getQueryMaster() { + return null; + } + + public abstract TajoConf getConf(); + + @Override + public ServiceTracker getServiceTracker() { + return null; + } + + @Override + public QueryMasterManagerService getQueryMasterManagerService() { + return null; + } + + @Override + public TaskRunnerManager getTaskRunnerManager() { + return null; + } + + @Override + public CatalogService getCatalog() { + return null; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + return null; + } + + @Override + public String getWorkerName() { + return null; + } + + @Override + public LocalDirAllocator getLocalDirAllocator() { + return null; + } + + @Override + public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { + return null; + } + + @Override + public TajoSystemMetrics getWorkerSystemMetrics() { + + if (tajoSystemMetrics == null) { + tajoSystemMetrics = new TajoSystemMetrics(getConf(), "test-file-group", "localhost"); + tajoSystemMetrics.start(); + } + return tajoSystemMetrics; + } + + @Override + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return null; + } + + @Override + public HistoryWriter getTaskHistoryWriter() { + return null; + } + + @Override + public HistoryReader getHistoryReader() { + return null; + } + + @Override + public void cleanup(String strPath) { + + } + + @Override + public void cleanupTemporalDirectories() { + + } + + @Override + public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { + + } + + @Override + public void setNumClusterNodes(int numClusterNodes) { + + } +} + diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java index 513eb692b7..65627c1767 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java @@ -104,13 +104,13 @@ public void testGet() throws IOException { @Test public void testAdjustFetchProcess() { - assertEquals(0.0f, Task.adjustFetchProcess(0, 0), 0); - assertEquals(0.0f, Task.adjustFetchProcess(10, 10), 0); - assertEquals(0.05f, Task.adjustFetchProcess(10, 9), 0); - assertEquals(0.1f, Task.adjustFetchProcess(10, 8), 0); - assertEquals(0.25f, Task.adjustFetchProcess(10, 5), 0); - assertEquals(0.45f, Task.adjustFetchProcess(10, 1), 0); - assertEquals(0.5f, Task.adjustFetchProcess(10, 0), 0); + assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(0, 0), 0); + assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(10, 10), 0); + assertEquals(0.05f, LegacyTaskImpl.adjustFetchProcess(10, 9), 0); + assertEquals(0.1f, LegacyTaskImpl.adjustFetchProcess(10, 8), 0); + assertEquals(0.25f, LegacyTaskImpl.adjustFetchProcess(10, 5), 0); + assertEquals(0.45f, LegacyTaskImpl.adjustFetchProcess(10, 1), 0); + assertEquals(0.5f, LegacyTaskImpl.adjustFetchProcess(10, 0), 0); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 7407acc1f9..2cee7d025f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -19,13 +19,15 @@ package org.apache.tajo.worker; import com.google.common.collect.Lists; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.tajo.*; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; -import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; @@ -42,9 +44,15 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; public class TestNodeResourceManager { - private NodeResourceManager resourceManager; - private MockNodeStatusUpdater statusUpdater; + private MockNodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private TajoWorker.WorkerContext workerContext; + + private CompositeService service; private int taskMemory; private TajoConf conf; @@ -61,29 +69,55 @@ public void setup() { conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); dispatcher = new AsyncDispatcher(); - dispatcher.init(conf); - dispatcher.start(); - - resourceManager = new NodeResourceManager(dispatcher); - resourceManager.init(conf); - resourceManager.start(); - - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); - statusUpdater.init(conf); - statusUpdater.start(); + taskDispatcher = new AsyncDispatcher(); + + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; + + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); + resourceManager = new MockNodeResourceManager(new Semaphore(0), dispatcher, taskDispatcher.getEventHandler()); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + }; + + service.init(conf); + service.start(); } @After public void tearDown() { - resourceManager.stop(); - statusUpdater.stop(); - dispatcher.stop(); + service.stop(); } @Test public void testNodeResourceAllocateEvent() throws Exception { int requestSize = 4; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -91,14 +125,14 @@ public void testNodeResourceAllocateEvent() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + // allocated all assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); } @@ -106,6 +140,7 @@ public void testNodeResourceAllocateEvent() throws Exception { public void testNodeResourceCancellation() throws Exception { int requestSize = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); int overSize = 10; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -113,18 +148,19 @@ public void testNodeResourceCancellation() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize + overSize)); + requestProto.addAllTaskRequest( + MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize + overSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); assertEquals(overSize, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); } @Test public void testNodeResourceDeallocateEvent() throws Exception { int requestSize = 4; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -132,21 +168,20 @@ public void testNodeResourceDeallocateEvent() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); //deallocate for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { // direct invoke handler for testing resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); } - assertEquals(0, resourceManager.getAllocatedSize()); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); } @@ -154,12 +189,38 @@ public void testNodeResourceDeallocateEvent() throws Exception { public void testParallelRequest() throws Exception { final int parallelCount = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES) * 2; final int taskSize = 100000; + resourceManager.setTaskHandlerEvent(true); + final AtomicInteger totalComplete = new AtomicInteger(); final AtomicInteger totalCanceled = new AtomicInteger(); final ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); - final Queue totalTasks = createTaskRequests(taskMemory, taskSize); + final Queue + totalTasks = MockNodeResourceManager.createTaskRequests(ebId, taskMemory, taskSize); + + // first request with starting ExecutionBlock + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + TaskAllocationRequestProto task = totalTasks.poll(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.addTaskRequest(task); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + CallFuture callFuture = new CallFuture(); + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + assertTrue(callFuture.get().getCancellationTaskCount() == 0); + totalComplete.incrementAndGet(); + // start parallel request ExecutorService executor = Executors.newFixedThreadPool(parallelCount); List futureList = Lists.newArrayList(); @@ -187,7 +248,6 @@ public void run() { totalCanceled.addAndGet(proto.getCancellationTaskCount()); } else { complete++; - dispatcher.getEventHandler().handle(new NodeResourceDeallocateEvent(task.getResource())); } } catch (Exception e) { fail(e.getMessage()); @@ -209,27 +269,4 @@ public void run() { executor.shutdown(); assertEquals(taskSize, totalComplete.get()); } - - protected static Queue createTaskRequests(int memory, int size) { - Queue requestProtoList = new LinkedBlockingQueue(); - for (int i = 0; i < size; i++) { - - ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0); - TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, i), 0); - - TajoWorkerProtocol.TaskRequestProto.Builder builder = - TajoWorkerProtocol.TaskRequestProto.newBuilder(); - builder.setId(taskAttemptId.getProto()); - builder.setShouldDie(true); - builder.setOutputTable(""); - builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); - builder.setClusteredOutput(false); - - - requestProtoList.add(TaskAllocationRequestProto.newBuilder() - .setResource(NodeResources.createResource(memory).getProto()) - .setTaskRequest(builder.build()).build()); - } - return requestProtoList; - } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index fb3c77e27c..af40554f31 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -22,6 +22,7 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.master.rm.Worker; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeStatusEvent; import org.junit.After; @@ -37,18 +38,36 @@ public class TestNodeStatusUpdater { private MockNodeStatusUpdater statusUpdater; private AsyncDispatcher dispatcher; private TajoConf conf; + private TajoWorker.WorkerContext workerContext; + @Before public void setup() { conf = new TajoConf(); conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL, 1000); dispatcher = new AsyncDispatcher(); dispatcher.init(conf); dispatcher.start(); - resourceManager = new NodeResourceManager(dispatcher); + resourceManager = new NodeResourceManager(dispatcher, null); resourceManager.init(conf); resourceManager.start(); } @@ -63,27 +82,25 @@ public void tearDown() { @Test(timeout = 20000) public void testNodeMembership() throws Exception { CountDownLatch barrier = new CountDownLatch(1); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); MockNodeStatusUpdater.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); barrier.await(); - assertTrue(resourceTracker.getTotalResource().containsKey(worker.getId())); + assertTrue(resourceTracker.getTotalResource().containsKey(workerContext.getConnectionInfo().getId())); assertEquals(resourceManager.getTotalResource(), - resourceTracker.getTotalResource().get(worker.getId())); + resourceTracker.getTotalResource().get(workerContext.getConnectionInfo().getId())); assertEquals(resourceManager.getAvailableResource(), - resourceTracker.getAvailableResource().get(worker.getId())); + resourceTracker.getAvailableResource().get(workerContext.getConnectionInfo().getId())); } @Test(timeout = 20000) public void testPing() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); @@ -100,16 +117,29 @@ public void testPing() throws Exception { @Test(timeout = 20000) public void testResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); + assertEquals(0, statusUpdater.getQueueSize()); for (int i = 0; i < statusUpdater.getQueueingLimit(); i++) { - dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, - resourceManager.getAvailableResource())); + dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); } barrier.await(); assertEquals(0, statusUpdater.getQueueSize()); } + + @Test(timeout = 20000) + public void testFlushResourceReport() throws Exception { + CountDownLatch barrier = new CountDownLatch(2); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); + statusUpdater.init(conf); + statusUpdater.start(); + + assertEquals(0, statusUpdater.getQueueSize()); + dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + + barrier.await(); + assertEquals(0, statusUpdater.getQueueSize()); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java new file mode 100644 index 0000000000..98b187b013 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -0,0 +1,330 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.*; +import org.apache.tajo.annotation.ThreadSafe; +import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.querymaster.QueryMaster; +import org.apache.tajo.querymaster.QueryMasterManagerService; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; +import org.apache.tajo.worker.event.ExecutionBlockStartEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Queue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.junit.Assert.*; + +public class TestTaskExecutor { + + private NodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; + private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private TajoWorker.WorkerContext workerContext; + + private CompositeService service; + private TajoConf conf; + private Semaphore barrier; + private Semaphore resourceManagerBarrier; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + dispatcher = new AsyncDispatcher(); + taskDispatcher = new AsyncDispatcher(); + + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; + + barrier = new Semaphore(0); + resourceManagerBarrier = new Semaphore(0); + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new TaskExecutor(barrier, taskManager, dispatcher.getEventHandler()); + resourceManager = new MockNodeResourceManager(resourceManagerBarrier, dispatcher, taskDispatcher.getEventHandler()); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } + }; + + service.init(conf); + service.start(); + } + + @After + public void tearDown() { + service.stop(); + } + + @Test + public void testTaskRequest() throws Exception { + int requestSize = 1; + + RunExecutionBlockRequestProto.Builder + ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + //verify running task + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(1, taskExecutor.getRunningTasks()); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(0, taskExecutor.getRunningTasks()); + assertEquals(1, taskExecutor.completeTasks); + + //verify the released resources + Thread.sleep(100); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + } + + @Test + public void testTaskException() throws Exception { + int requestSize = 1; + + RunExecutionBlockRequestProto.Builder + ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost()+":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); + + taskExecutor.throwException.set(true); + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + //verify running task + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(1, taskExecutor.getRunningTasks()); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(0, taskExecutor.getRunningTasks()); + assertEquals(0, taskExecutor.completeTasks); + + //verify the released resources + Thread.sleep(100); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + } + + class TaskExecutor extends MockTaskExecutor { + int completeTasks; + AtomicBoolean throwException = new AtomicBoolean(); + + public TaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { + super(barrier, taskManager, rmEventHandler); + } + + @Override + protected void stopTask(TaskAttemptId taskId) { + super.stopTask(taskId); + super.barrier.release(); + } + + @Override + protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null); + + return new Task() { + @Override + public void init() throws IOException { + + try { + Thread.sleep(50); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + @Override + public void fetch() { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + @Override + public void run() throws Exception { + Thread.sleep(50); + + if(throwException.get()) throw new RuntimeException(); + + taskAttemptContext.stop(); + taskAttemptContext.setProgress(1.0f); + taskAttemptContext.setState(TajoProtos.TaskAttemptState.TA_SUCCEEDED); + completeTasks++; + } + + @Override + public void kill() { + + } + + @Override + public void abort() { + + } + + @Override + public void cleanup() { + } + + @Override + public boolean hasFetchPhase() { + return false; + } + + @Override + public boolean isProgressChanged() { + return false; + } + + @Override + public boolean isStopped() { + return taskAttemptContext.isStopped(); + } + + @Override + public void updateProgress() { + + } + + @Override + public TaskAttemptContext getTaskContext() { + return taskAttemptContext; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return context; + } + + @Override + public TajoWorkerProtocol.TaskStatusProto getReport() { + TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + builder.setWorkerName("localhost:0"); + builder.setId(taskAttemptContext.getTaskId().getProto()) + .setProgress(taskAttemptContext.getProgress()) + .setState(taskAttemptContext.getState()); + + builder.setInputStats(new TableStats().getProto()); + return builder.build(); + } + }; + } + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java new file mode 100644 index 0000000000..8bca489e20 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -0,0 +1,185 @@ +/** + * 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.conf.Configuration; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.tajo.*; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.worker.event.ExecutionBlockStartEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.concurrent.*; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.junit.Assert.*; + +public class TestTaskManager { + + private NodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; + private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private TajoWorker.WorkerContext workerContext; + + private CompositeService service; + private int taskMemory; + private TajoConf conf; + private Semaphore barrier; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + + taskMemory = 512; + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, + taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); + + dispatcher = new AsyncDispatcher(); + taskDispatcher = new AsyncDispatcher(); + + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; + barrier = new Semaphore(0); + taskManager = new MockTaskManager(barrier, taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); + resourceManager = new NodeResourceManager(dispatcher, taskDispatcher.getEventHandler()); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } + }; + + service.init(conf); + service.start(); + } + + @After + public void tearDown() { + service.stop(); + } + + @Test(timeout = 10000) + public void testExecutionBlockStart() throws Exception { + int requestSize = 1; + + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNotNull(taskManager.getExecutionBlockContext(ebId)); + assertEquals(ebId, taskManager.getExecutionBlockContext(ebId).getExecutionBlockId()); + } + + @Test(timeout = 10000) + public void testExecutionBlockStop() throws Exception { + + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost()+":" + + workerContext.getConnectionInfo().getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + taskDispatcher.getEventHandler().handle(new ExecutionBlockStartEvent(ebRequestProto.build())); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNotNull(taskManager.getExecutionBlockContext(ebId)); + assertEquals(ebId, taskManager.getExecutionBlockContext(ebId).getExecutionBlockId()); + + ExecutionBlockListProto.Builder ebList = ExecutionBlockListProto.newBuilder(); + taskDispatcher.getEventHandler().handle(new ExecutionBlockStopEvent(ebId.getProto(), ebList.build())); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNull(taskManager.getExecutionBlockContext(ebId)); + } +} From 91319ee3eed1c950551071eca10a6587f635a48e Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 8 Jun 2015 16:57:23 +0900 Subject: [PATCH 02/80] TAJO-1397 --- .../java/org/apache/tajo/conf/TajoConf.java | 5 +- .../main/java/org/apache/tajo/util/TUtil.java | 10 ++ .../apache/tajo/master/QueryInProgress.java | 6 +- .../org/apache/tajo/master/QueryManager.java | 21 ++- .../org/apache/tajo/master/TajoMaster.java | 10 +- .../apache/tajo/master/rm/TajoRMContext.java | 2 +- ...eManager.java => TajoResourceManager.java} | 78 +++++++---- .../tajo/master/rm/TajoResourceTracker.java | 32 +++-- .../org/apache/tajo/master/rm/Worker.java | 129 ++++++++---------- .../tajo/master/rm/WorkerResourceManager.java | 3 +- .../tajo/master/rm/WorkerStatusEvent.java | 26 ++-- .../tajo/querymaster/QueryMasterTask.java | 4 +- .../org/apache/tajo/worker/TajoWorker.java | 5 +- .../main/proto/QueryCoordinatorProtocol.proto | 58 ++++---- .../main/proto/ResourceTrackerProtocol.proto | 8 -- .../org/apache/tajo/TajoTestingCluster.java | 6 +- .../master/rm/TestTajoResourceManager.java | 90 ++++++------ 17 files changed, 249 insertions(+), 244 deletions(-) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{TajoWorkerResourceManager.java => TajoResourceManager.java} (90%) 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 3f350c3143..90212f220e 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 @@ -155,7 +155,6 @@ public static enum ConfVars implements ConfigKey { RESOURCE_TRACKER_HEARTBEAT_TIMEOUT("tajo.resource-tracker.heartbeat.timeout-secs", 120 * 1000), // seconds // QueryMaster resource - TAJO_QUERYMASTER_DISK_SLOT("tajo.qm.resource.disk.slots", 0.0f, Validators.min("0.0f")), TAJO_QUERYMASTER_MEMORY_MB("tajo.qm.resource.memory-mb", 512, Validators.min("64")), TAJO_QUERYMASTER_ALLOCATION_TIMEOUT("tajo.qm.resource.allocation.timeout", "3 sec"), @@ -193,8 +192,8 @@ public static enum ConfVars implements ConfigKey { WORKER_HEARTBEAT_INTERVAL("tajo.worker.heartbeat.interval", 10 * 1000), // 10 sec - // Resource Manager - RESOURCE_MANAGER_CLASS("tajo.resource.manager", "org.apache.tajo.master.rm.TajoWorkerResourceManager", + // Resource Scheduler + RESOURCE_SCHEDULER_CLASS("tajo.resource.scheduler", "org.apache.tajo.master.scheduler.SimpleScheduler", Validators.groups(Validators.notNull(), Validators.clazz())), // Catalog 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 2293ef5488..0a77e5ca36 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 @@ -19,6 +19,7 @@ package org.apache.tajo.util; import com.google.common.base.Objects; +import org.apache.tajo.conf.TajoConf; import java.lang.reflect.Array; import java.util.*; @@ -267,4 +268,13 @@ public static String getCurrentCodePoint(final int depth) { StackTraceElement element = ste[2 + depth]; return element.getClassName() + ":" + element.getMethodName() + "(" + element.getLineNumber() +")"; } + + public static T checkTypeAndGet(Object instance, Class type) { + if (!type.isInstance(instance)) { + throw new IllegalArgumentException(instance.getClass().getSimpleName() + + " must be a " + type.getSimpleName() + " type"); + + } + return (T) instance; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java index 6a074a2f57..f0dc0042ac 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java @@ -30,6 +30,7 @@ import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol.QueryExecutionRequestProto; +import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.master.rm.WorkerResourceManager; import org.apache.tajo.plan.logical.LogicalRootNode; import org.apache.tajo.rpc.*; @@ -131,8 +132,9 @@ public boolean startQueryMaster() { } try { LOG.info("Initializing QueryInProgress for QueryID=" + queryId); - WorkerResourceManager resourceManager = masterContext.getResourceManager(); - WorkerAllocatedResource resource = resourceManager.allocateQueryMaster(this); + TajoResourceManager resourceManager = masterContext.getResourceManager(); + resourceManager.getScheduler().getq + //WorkerAllocatedResource resource = resourceManager.allocateQueryMaster(this); // if no resource to allocate a query master if(resource == null) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java index 0c8d8cea50..daba7f7227 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java @@ -35,6 +35,7 @@ import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.master.scheduler.QuerySchedulingInfo; import org.apache.tajo.master.scheduler.SimpleFifoScheduler; import org.apache.tajo.plan.logical.LogicalRootNode; import org.apache.tajo.querymaster.QueryJobEvent; @@ -178,12 +179,13 @@ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, Strin QueryInProgress queryInProgress = new QueryInProgress(masterContext, session, queryContext, queryId, sql, jsonExpr, plan); - synchronized (submittedQueries) { - queryInProgress.getQueryInfo().setQueryMaster(""); - submittedQueries.put(queryInProgress.getQueryId(), queryInProgress); - } + queryInProgress.getQueryInfo().setQueryMaster(""); + submittedQueries.put(queryInProgress.getQueryId(), queryInProgress); - scheduler.addQuery(queryInProgress); + //scheduler.addQuery(queryInProgress); + QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo(queryInProgress.getQueryId(), 1, + queryInProgress.getQueryInfo().getStartTime()); + masterContext.getResourceManager().submitQuery(querySchedulingInfo); return queryInProgress.getQueryInfo(); } @@ -191,13 +193,8 @@ public QueryInfo startQueryJob(QueryId queryId) throws Exception { QueryInProgress queryInProgress; - synchronized (submittedQueries) { - queryInProgress = submittedQueries.remove(queryId); - } - - synchronized (runningQueries) { - runningQueries.put(queryInProgress.getQueryId(), queryInProgress); - } + queryInProgress = submittedQueries.remove(queryId); + runningQueries.put(queryInProgress.getQueryId(), queryInProgress); if (queryInProgress.startQueryMaster()) { dispatcher.getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index c41fdde0d9..1c731890c2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -39,7 +39,7 @@ import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.engine.function.FunctionLoader; import org.apache.tajo.function.FunctionSignature; -import org.apache.tajo.master.rm.TajoWorkerResourceManager; +import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.master.rm.WorkerResourceManager; import org.apache.tajo.metrics.CatalogMetricsGaugeSet; import org.apache.tajo.metrics.WorkerResourceMetricsGaugeSet; @@ -121,7 +121,7 @@ public class TajoMaster extends CompositeService { private QueryCoordinatorService tajoMasterService; private SessionManager sessionManager; - private WorkerResourceManager resourceManager; + private TajoResourceManager resourceManager; //Web Server private StaticHttpServer webServer; private TajoRestService restServer; @@ -230,9 +230,9 @@ private void initSystemMetrics() { private void initResourceManager() throws Exception { Class resourceManagerClass = (Class) - systemConf.getClass(ConfVars.RESOURCE_MANAGER_CLASS.varname, TajoWorkerResourceManager.class); + systemConf.getClass(ConfVars.RESOURCE_MANAGER_CLASS.varname, TajoResourceManager.class); Constructor constructor = resourceManagerClass.getConstructor(MasterContext.class); - resourceManager = constructor.newInstance(context); + resourceManager = new TajoResourceManager(context); addIfService(resourceManager); } @@ -457,7 +457,7 @@ public QueryManager getQueryJobManager() { return queryManager; } - public WorkerResourceManager getResourceManager() { + public TajoResourceManager getResourceManager() { return resourceManager; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java index bb8cc126b7..3c8f8b0626 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java @@ -29,7 +29,7 @@ import java.util.concurrent.ConcurrentMap; /** - * It's a worker resource manager context. It contains all context data about TajoWorkerResourceManager. + * It's a worker resource manager context. It contains all context data about TajoResourceManager. */ public class TajoRMContext { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java similarity index 90% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java index 90a4eb505a..1d4c85bb6a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java @@ -34,13 +34,20 @@ import org.apache.tajo.QueryId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.ContainerProtocol; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; import org.apache.tajo.master.QueryInProgress; +import org.apache.tajo.master.QueryInfo; import org.apache.tajo.master.TajoMaster; +import org.apache.tajo.master.scheduler.QuerySchedulingInfo; +import org.apache.tajo.master.scheduler.SimpleScheduler; +import org.apache.tajo.master.scheduler.TajoScheduler; +import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.CancelableRpcCallback; import org.apache.tajo.rpc.RpcUtils; import org.apache.tajo.util.ApplicationIdUtils; import org.apache.tajo.util.BasicFuture; +import org.apache.tajo.util.TUtil; import java.io.IOException; import java.util.*; @@ -52,9 +59,9 @@ /** * It manages all resources of tajo workers. */ -public class TajoWorkerResourceManager extends CompositeService implements WorkerResourceManager { +public class TajoResourceManager extends CompositeService { /** class logger */ - private static final Log LOG = LogFactory.getLog(TajoWorkerResourceManager.class); + private static final Log LOG = LogFactory.getLog(TajoResourceManager.class); static AtomicInteger containerIdSeq = new AtomicInteger(0); @@ -79,6 +86,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke private AtomicBoolean stopped = new AtomicBoolean(false); private TajoConf systemConf; + private TajoScheduler scheduler; private ConcurrentMap allocatedResourceMap = Maps .newConcurrentMap(); @@ -86,19 +94,18 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke /** It receives status messages from workers and their resources. */ private TajoResourceTracker resourceTracker; - public TajoWorkerResourceManager(TajoMaster.MasterContext masterContext) { - super(TajoWorkerResourceManager.class.getSimpleName()); + public TajoResourceManager(TajoMaster.MasterContext masterContext) { + super(TajoResourceManager.class.getSimpleName()); this.masterContext = masterContext; } - public TajoWorkerResourceManager(TajoConf systemConf) { - super(TajoWorkerResourceManager.class.getSimpleName()); + public TajoResourceManager(TajoConf systemConf) { + super(TajoResourceManager.class.getSimpleName()); } @Override public void serviceInit(Configuration conf) throws Exception { - Preconditions.checkArgument(conf instanceof TajoConf); - this.systemConf = (TajoConf) conf; + this.systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); AsyncDispatcher dispatcher = new AsyncDispatcher(); addIfService(dispatcher); @@ -119,6 +126,9 @@ public void serviceInit(Configuration conf) throws Exception { resourceTracker = new TajoResourceTracker(this, workerLivelinessMonitor); addIfService(resourceTracker); + //TODO configuable + scheduler = new SimpleScheduler(rmContext); + addIfService(scheduler); super.serviceInit(systemConf); } @@ -145,26 +155,25 @@ public void handle(WorkerEvent event) { } } - @Override + @Deprecated public Map getWorkers() { return ImmutableMap.copyOf(rmContext.getWorkers()); } - - @Override + @Deprecated public Map getInactiveWorkers() { return ImmutableMap.copyOf(rmContext.getInactiveWorkers()); } - + @Deprecated public Collection getQueryMasters() { return Collections.unmodifiableSet(rmContext.getQueryMasterWorker()); } @Override public void serviceStop() throws Exception { - if(stopped.get()) { + if(stopped.getAndSet(true)) { return; } - stopped.set(true); + if(workerResourceAllocator != null) { workerResourceAllocator.interrupt(); } @@ -176,7 +185,6 @@ public void serviceStop() throws Exception { * * @return The prefix of queryId. It is generated when a TajoMaster starts up. */ - @Override public String getSeedQueryId() throws IOException { return queryIdSeed; } @@ -186,31 +194,41 @@ TajoResourceTracker getResourceTracker() { return resourceTracker; } - private WorkerResourceAllocationRequest createQMResourceRequest(QueryId queryId) { - float queryMasterDefaultDiskSlot = masterContext.getConf().getFloatVar( - TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT); + public TajoScheduler getScheduler() { + return scheduler; + } + + private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceRequest(QueryInfo queryInfo) { int queryMasterDefaultMemoryMB = masterContext.getConf().getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB); - WorkerResourceAllocationRequest.Builder builder = WorkerResourceAllocationRequest.newBuilder(); - builder.setQueryId(queryId.getProto()); - builder.setMaxMemoryMBPerContainer(queryMasterDefaultMemoryMB); - builder.setMinMemoryMBPerContainer(queryMasterDefaultMemoryMB); - builder.setMaxDiskSlotPerContainer(queryMasterDefaultDiskSlot); - builder.setMinDiskSlotPerContainer(queryMasterDefaultDiskSlot); - builder.setResourceRequestPriority(ResourceRequestPriority.MEMORY); - builder.setNumContainers(1); + QueryCoordinatorProtocol.NodeResourceRequestProto.Builder builder = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); + builder.setQueryId(queryInfo.getQueryId().getProto()) + .setAllocation(NodeResources.createResource(queryMasterDefaultMemoryMB).getProto()) + .setType(QueryCoordinatorProtocol.ResourceType.QUERYMASTER) + .setPriority(1) + .setNumContainers(1) + .setUserId(queryInfo.getQueryContext().getUser()); + //TODO .setQueue(queryInfo.getQueue()); return builder.build(); } - @Override - public WorkerAllocatedResource allocateQueryMaster(QueryInProgress queryInProgress) { + /** + * Submit a query to scheduler + * + * @param queryInProgress QueryInProgress + * @return boolean + */ + public boolean submitQuery(QuerySchedulingInfo schedulingInfo) { // 3 seconds, by default long timeout = masterContext.getConf().getTimeVar( TajoConf.ConfVars.TAJO_QUERYMASTER_ALLOCATION_TIMEOUT, TimeUnit.MILLISECONDS); // Create a resource request for a query master - WorkerResourceAllocationRequest qmResourceRequest = createQMResourceRequest(queryInProgress.getQueryId()); + + List allocation = + scheduler.submitQuery(schedulingInfo.getQueryId(), createQMResourceRequest(queryInProgress.getQueryInfo())); + // call future for async call final CancelableRpcCallback callFuture = @@ -314,7 +332,7 @@ public void run() { ", liveWorkers=" + rmContext.getWorkers().size()); } - // TajoWorkerResourceManager can't return allocated disk slots occasionally. + // TajoResourceManager can't return allocated disk slots occasionally. // Because the rest resource request can remains after QueryMaster stops. // Thus we need to find whether QueryId stopped or not. if (!rmContext.getStoppedQueryIds().contains(resourceRequest.queryId)) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index 2a18de7ecb..acdcd0ee14 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -20,16 +20,16 @@ import com.google.protobuf.RpcCallback; 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.service.AbstractService; -import org.apache.tajo.common.exception.NotImplementedException; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeatResponse; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.util.NetUtils; import org.apache.tajo.util.ProtoUtil; @@ -37,8 +37,7 @@ import java.io.IOError; import java.net.InetSocketAddress; -import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.NodeHeartbeat; -import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService; +import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; /** * It receives pings that workers periodically send. The ping messages contains the worker resources and their statuses. @@ -59,8 +58,8 @@ public class TajoResourceTracker extends AbstractService implements TajoResource /** Class logger */ private Log LOG = LogFactory.getLog(TajoResourceTracker.class); - private final WorkerResourceManager manager; - /** the context of TajoWorkerResourceManager */ + private final TajoResourceManager manager; + /** the context of TajoResourceManager */ private final TajoRMContext rmContext; /** Liveliness monitor which checks ping expiry times of workers */ private final WorkerLivelinessMonitor workerLivelinessMonitor; @@ -70,7 +69,7 @@ public class TajoResourceTracker extends AbstractService implements TajoResource /** The bind address of RPC server of worker resource tracker */ private InetSocketAddress bindAddress; - public TajoResourceTracker(WorkerResourceManager manager, WorkerLivelinessMonitor workerLivelinessMonitor) { + public TajoResourceTracker(TajoResourceManager manager, WorkerLivelinessMonitor workerLivelinessMonitor) { super(TajoResourceTracker.class.getSimpleName()); this.manager = manager; this.rmContext = manager.getRMContext(); @@ -117,10 +116,10 @@ public void serviceStop() throws Exception { private static final TajoHeartbeatResponse.Builder builder = TajoHeartbeatResponse.newBuilder().setHeartbeatResult(ProtoUtil.TRUE); - private static WorkerStatusEvent createStatusEvent(int workerId, NodeHeartbeat heartbeat) { + private static WorkerStatusEvent createStatusEvent(int workerId, NodeHeartbeatRequestProto heartbeat) { return new WorkerStatusEvent( workerId, - heartbeat.getServerStatus().getRunningTaskNum(), + heartbeat.ggetServerStatus().getRunningTaskNum(), heartbeat.getServerStatus().getJvmHeap().getMaxHeap(), heartbeat.getServerStatus().getJvmHeap().getFreeHeap(), heartbeat.getServerStatus().getJvmHeap().getTotalHeap()); @@ -129,17 +128,20 @@ private static WorkerStatusEvent createStatusEvent(int workerId, NodeHeartbeat h @Override public void heartbeat( RpcController controller, - NodeHeartbeat heartbeat, - RpcCallback done) { + NodeHeartbeatRequestProto heartbeat, + RpcCallback done) { try { // get a workerId from the heartbeat - int workerId = heartbeat.getConnectionInfo().getId(); + int workerId = heartbeat.getWorkerId(); if(rmContext.getWorkers().containsKey(workerId)) { // if worker is running - // status update - rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(workerId, heartbeat)); + if (heartbeat.hasAvailableResource()) { + // status update + rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(heartbeat)); + } + // refresh ping workerLivelinessMonitor.receivedPing(workerId); @@ -185,7 +187,7 @@ public void heartbeat( } @Override - public void nodeHeartbeat(RpcController controller, TajoResourceTrackerProtocol.NodeHeartbeatRequestProto request, + public void nodeHeartbeat(RpcController controller, NodeHeartbeatRequestProto request, RpcCallback done) { //TODO implement with ResourceManager for scheduler TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.Builder diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java index 6535688d56..0a38a191b2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java @@ -18,13 +18,19 @@ package org.apache.tajo.master.rm; +import io.netty.util.internal.PlatformDependent; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.*; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.util.TUtil; import java.util.EnumSet; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.locks.ReentrantReadWriteLock; /** @@ -34,17 +40,34 @@ public class Worker implements EventHandler, Comparable { /** class logger */ private static final Log LOG = LogFactory.getLog(Worker.class); - private final ReentrantReadWriteLock.ReadLock readLock; - private final ReentrantReadWriteLock.WriteLock writeLock; - - /** context of {@link org.apache.tajo.master.rm.TajoWorkerResourceManager} */ + /** context of {@link TajoResourceManager} */ private final TajoRMContext rmContext; /** last heartbeat time */ - private long lastHeartbeatTime; + private volatile long lastHeartbeatTime; + + private volatile int numRunningTasks; + + private volatile int numRunningQueryMaster; + + private static AtomicLongFieldUpdater HEARTBEAT_TIME_UPDATER; + private static AtomicIntegerFieldUpdater RUNNING_TASK_UPDATER; + private static AtomicIntegerFieldUpdater RUNNING_QM_UPDATER; + + static { + HEARTBEAT_TIME_UPDATER = PlatformDependent.newAtomicLongFieldUpdater(Worker.class, "lastHeartbeatTime"); + if (HEARTBEAT_TIME_UPDATER == null) { + HEARTBEAT_TIME_UPDATER = AtomicLongFieldUpdater.newUpdater(NodeResource.class, "lastHeartbeatTime"); + RUNNING_TASK_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeResource.class, "numRunningTasks"); + RUNNING_QM_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeResource.class, "numRunningQueryMaster"); + } else { + RUNNING_TASK_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeResource.class, "numRunningTasks"); + RUNNING_QM_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeResource.class, "numRunningQueryMaster"); + } + } /** Resource capability */ - private final WorkerResource resource; + private final NodeResource resource; /** Worker connection information */ private WorkerConnectionInfo connectionInfo; @@ -91,16 +114,12 @@ public class Worker implements EventHandler, Comparable { private final StateMachine stateMachine = stateMachineFactory.make(this, WorkerState.NEW); - public Worker(TajoRMContext rmContext, WorkerResource resource, WorkerConnectionInfo connectionInfo) { + public Worker(TajoRMContext rmContext, NodeResource resource, WorkerConnectionInfo connectionInfo) { this.rmContext = rmContext; this.connectionInfo = connectionInfo; this.lastHeartbeatTime = System.currentTimeMillis(); this.resource = resource; - - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - this.readLock = lock.readLock(); - this.writeLock = lock.writeLock(); } public int getWorkerId() { @@ -112,23 +131,19 @@ public WorkerConnectionInfo getConnectionInfo() { } public void setLastHeartbeatTime(long lastheartbeatReportTime) { - this.writeLock.lock(); + HEARTBEAT_TIME_UPDATER.lazySet(this, lastheartbeatReportTime); + } - try { - this.lastHeartbeatTime = lastheartbeatReportTime; - } finally { - this.writeLock.unlock(); - } + public void setNumRunningQueryMaster(int numRunningQueryMaster) { + RUNNING_QM_UPDATER.lazySet(this, numRunningQueryMaster); } - public long getLastHeartbeatTime() { - this.readLock.lock(); + public void setNumRunningTasks(int numRunningTasks) { + RUNNING_TASK_UPDATER.lazySet(this, numRunningTasks); + } - try { - return this.lastHeartbeatTime; - } finally { - this.readLock.unlock(); - } + public long getLastHeartbeatTime() { + return this.lastHeartbeatTime; } /** @@ -136,20 +151,14 @@ public long getLastHeartbeatTime() { * @return the current state of worker */ public WorkerState getState() { - this.readLock.lock(); - - try { - return this.stateMachine.getCurrentState(); - } finally { - this.readLock.unlock(); - } + return this.stateMachine.getCurrentState(); } /** * * @return the current resource capability of worker */ - public WorkerResource getResource() { + public NodeResource getResource() { return this.resource; } @@ -171,19 +180,17 @@ public boolean equals(Object o) { if (lastHeartbeatTime != worker.lastHeartbeatTime) return false; if (connectionInfo != null ? !connectionInfo.equals(worker.connectionInfo) : worker.connectionInfo != null) return false; - if (readLock != null ? !readLock.equals(worker.readLock) : worker.readLock != null) return false; + if (resource != null ? !resource.equals(worker.resource) : worker.resource != null) return false; if (rmContext != null ? !rmContext.equals(worker.rmContext) : worker.rmContext != null) return false; if (stateMachine != null ? !stateMachine.equals(worker.stateMachine) : worker.stateMachine != null) return false; - if (writeLock != null ? !writeLock.equals(worker.writeLock) : worker.writeLock != null) return false; return true; } @Override public int hashCode() { - int result = readLock != null ? readLock.hashCode() : 0; - result = 31 * result + (writeLock != null ? writeLock.hashCode() : 0); + int result = 0; result = 31 * result + (rmContext != null ? rmContext.hashCode() : 0); result = 31 * result + (int) (lastHeartbeatTime ^ (lastHeartbeatTime >>> 32)); result = 31 * result + (resource != null ? resource.hashCode() : 0); @@ -206,10 +213,8 @@ public static class StatusUpdateTransition implements @Override public WorkerState transition(Worker worker, WorkerEvent event) { - if (!(event instanceof WorkerStatusEvent)) { - throw new IllegalArgumentException("event should be a WorkerStatusEvent type."); - } - WorkerStatusEvent statusEvent = (WorkerStatusEvent) event; + + WorkerStatusEvent statusEvent = TUtil.checkTypeAndGet(event, WorkerStatusEvent.class); worker.updateStatus(statusEvent); return WorkerState.RUNNING; @@ -217,17 +222,10 @@ public WorkerState transition(Worker worker, WorkerEvent event) { } private void updateStatus(WorkerStatusEvent statusEvent) { - this.writeLock.lock(); - - try { - lastHeartbeatTime = System.currentTimeMillis(); - resource.setNumRunningTasks(statusEvent.getRunningTaskNum()); - resource.setMaxHeap(statusEvent.maxHeap()); - resource.setFreeHeap(statusEvent.getFreeHeap()); - resource.setTotalHeap(statusEvent.getTotalHeap()); - } finally { - this.writeLock.unlock(); - } + setLastHeartbeatTime(System.currentTimeMillis()); + setNumRunningTasks(statusEvent.getRunningTaskNum()); + setNumRunningQueryMaster(statusEvent.getRunningQMNum()); + NodeResources.update(resource, statusEvent.getResource()); } public static class DeactivateNodeTransition implements SingleArcTransition { @@ -265,26 +263,19 @@ public void transition(Worker worker, WorkerEvent workerEvent) { @Override public void handle(WorkerEvent event) { LOG.debug("Processing " + event.getWorkerId() + " of type " + event.getType()); + WorkerState oldState = getState(); try { - writeLock.lock(); - WorkerState oldState = getState(); - try { - stateMachine.doTransition(event.getType(), event); - } catch (InvalidStateTransitonException e) { - LOG.error("Can't handle this event at current state" - + ", eventType:" + event.getType().name() - + ", oldState:" + oldState.name() - + ", nextState:" + getState().name() - , e); - LOG.error("Invalid event " + event.getType() + " on Worker " + getWorkerId()); - } - if (oldState != getState()) { - LOG.info(getWorkerId() + " Node Transitioned from " + oldState + " to " + getState()); - } + stateMachine.doTransition(event.getType(), event); + } catch (InvalidStateTransitonException e) { + LOG.error("Can't handle this event at current state" + + ", eventType:" + event.getType().name() + + ", oldState:" + oldState.name() + + ", nextState:" + getState().name() + , e); + LOG.error("Invalid event " + event.getType() + " on Worker " + getWorkerId()); } - - finally { - writeLock.unlock(); + if (oldState != getState()) { + LOG.info(getWorkerId() + " Node Transitioned from " + oldState + " to " + getState()); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java index 3d5e062bef..054de1c905 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java @@ -24,7 +24,6 @@ import org.apache.tajo.ipc.ContainerProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol.ClusterResourceSummary; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.WorkerResourceAllocationRequest; import org.apache.tajo.master.QueryInProgress; import java.io.IOException; @@ -54,7 +53,7 @@ public interface WorkerResourceManager extends Service { * @param request Request description * @param rpcCallBack Callback function */ - public void allocateWorkerResources(WorkerResourceAllocationRequest request, + public void allocateWorkerResources(QueryCoordinatorProtocol.NodeResourceRequestProto request, RpcCallback rpcCallBack); /** diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java index f1ab40158b..c117aecff2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java @@ -18,37 +18,33 @@ package org.apache.tajo.master.rm; +import org.apache.tajo.resource.NodeResource; + /** * {@link TajoResourceTracker} produces this event, and its destination is * {@link org.apache.tajo.master.rm.Worker.StatusUpdateTransition} of {@link Worker}. */ public class WorkerStatusEvent extends WorkerEvent { private final int runningTaskNum; - private final long maxHeap; - private final long freeHeap; - private final long totalHeap; + private final int runningQMNum; + private final NodeResource resource; - public WorkerStatusEvent(int workerId, int runningTaskNum, long maxHeap, long freeHeap, long totalHeap) { + public WorkerStatusEvent(int workerId, int runningTaskNum, int runningQMNum, NodeResource resource) { super(workerId, WorkerEventType.STATE_UPDATE); this.runningTaskNum = runningTaskNum; - this.maxHeap = maxHeap; - this.freeHeap = freeHeap; - this.totalHeap = totalHeap; + this.runningQMNum = runningQMNum; + this.resource = resource; } public int getRunningTaskNum() { return runningTaskNum; } - public long maxHeap() { - return maxHeap; - } - - public long getFreeHeap() { - return freeHeap; + public int getRunningQMNum() { + return runningQMNum; } - public long getTotalHeap() { - return totalHeap; + public NodeResource getResource() { + return resource; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index 2809a706b9..4e9554eed4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -43,7 +43,7 @@ import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.TajoContainerProxy; import org.apache.tajo.master.event.*; -import org.apache.tajo.master.rm.TajoWorkerResourceManager; +import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.plan.LogicalOptimizer; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.LogicalPlanner; @@ -146,7 +146,7 @@ public void init(Configuration conf) { queryTaskContext = new QueryMasterTaskContext(); String resourceManagerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_MANAGER_CLASS); - if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) { + if(resourceManagerClassName.indexOf(TajoResourceManager.class.getName()) >= 0) { resourceAllocator = new TajoResourceAllocator(queryTaskContext); } else { throw new UnimplementedException(resourceManagerClassName + " is not supported yet"); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index fbd070e5d0..010a11d40c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -40,12 +40,11 @@ import org.apache.tajo.rpc.RpcClientManager; import org.apache.tajo.rpc.RpcConstants; import org.apache.tajo.service.ServiceTracker; -import org.apache.tajo.service.ServiceTrackerException; import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.service.TajoMasterInfo; import org.apache.tajo.ipc.QueryCoordinatorProtocol.ClusterResourceSummary; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.rm.TajoWorkerResourceManager; +import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.querymaster.QueryMaster; import org.apache.tajo.querymaster.QueryMasterManagerService; @@ -169,7 +168,7 @@ public void serviceInit(Configuration conf) throws Exception { String resourceManagerClassName = systemConf.getVar(ConfVars.RESOURCE_MANAGER_CLASS); boolean randomPort = true; - if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) { + if(resourceManagerClassName.indexOf(TajoResourceManager.class.getName()) >= 0) { randomPort = false; } diff --git a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto index 2440e2af5a..1e25647be5 100644 --- a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto +++ b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto @@ -60,6 +60,7 @@ message ServerStatusProto { required JvmHeap jvmHeap = 6; } +//deplecated message TajoHeartbeat { required WorkerConnectionInfoProto connectionInfo = 1; optional QueryIdProto queryId = 2; @@ -68,7 +69,7 @@ message TajoHeartbeat { optional string statusMessage = 5; optional float queryProgress = 6; } - +//deplecated message TajoHeartbeatResponse { message ResponseCommand { required string command = 1; @@ -90,33 +91,12 @@ message ClusterResourceSummary { required int32 totalAvailableMemoryMB = 7; } -enum ResourceRequestPriority { - MEMORY = 1; - DISK = 2; -} - -message WorkerResourceAllocationRequest { - required QueryIdProto queryId = 1; - required ResourceRequestPriority resourceRequestPriority = 2; - - required int32 numContainers = 3; - - required int32 maxMemoryMBPerContainer = 4; - required int32 minMemoryMBPerContainer = 5; - - required float maxDiskSlotPerContainer = 6; - required float minDiskSlotPerContainer = 7; -} - -message WorkerResourceProto { - required WorkerConnectionInfoProto connectionInfo = 1; - required int32 memoryMB = 2 ; - required float diskSlots = 3; +enum ResourceType { + LEAF = 1; + INTERMEDIATE = 2; + QUERYMASTER = 3; } -message WorkerResourcesRequest { - repeated WorkerResourceProto workerResources = 1; -} message WorkerResourceReleaseRequest { required ExecutionBlockIdProto executionBlockId = 1; @@ -136,9 +116,31 @@ message WorkerResourceAllocationResponse { repeated WorkerAllocatedResource workerAllocatedResource = 2; } +/////////////////// +message AllocationResourceProto { + required int32 workerId = 1; + required NodeResourceProto resource = 2; +} + +message NodeResourceRequestProto { + optional string queue = 1; + required string userId = 2; + required ResourceType type = 3; + required int32 priority = 4; + required QueryIdProto queryId = 5; + required int32 numContainers = 6; + required NodeResourceProto allocation = 7; + repeated int32 candidateNodes = 8; +} + +message NodeResourceResponseProto { + required QueryIdProto queryId = 1; + repeated AllocationResourceProto resource = 2; +} +/////////////////// service QueryCoordinatorProtocolService { rpc heartbeat(TajoHeartbeat) returns (TajoHeartbeatResponse); - rpc allocateWorkerResources(WorkerResourceAllocationRequest) returns (WorkerResourceAllocationResponse); + rpc allocateNodeResources(NodeResourceRequestProto) returns (NodeResourceResponseProto); rpc releaseWorkerResource(WorkerResourceReleaseRequest) returns (BoolProto); - rpc getAllWorkerResource(NullProto) returns (WorkerResourcesRequest); + rpc getAllWorkerResource(NullProto) returns (NodeResourceProto); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto index dffd8c9ee4..84d4c08ab5 100644 --- a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto +++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto @@ -29,13 +29,6 @@ import "TajoIdProtos.proto"; package hadoop.yarn; -// deprecated -message NodeHeartbeat { - required WorkerConnectionInfoProto connectionInfo = 1; - optional ServerStatusProto serverStatus = 2; - optional string statusMessage = 3; -} - message NodeHeartbeatRequestProto { required int32 workerId = 1; optional NodeResourceProto totalResource = 2; @@ -61,6 +54,5 @@ message NodeStatusProto { } service TajoResourceTrackerProtocolService { - rpc heartbeat(NodeHeartbeat) returns (TajoHeartbeatResponse); rpc nodeHeartbeat(NodeHeartbeatRequestProto) returns (NodeHeartbeatResponseProto); } \ No newline at end of file diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 9b5980b2e8..dd54962d8b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -33,7 +33,6 @@ import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.apache.tajo.catalog.*; -import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.client.TajoClient; import org.apache.tajo.client.TajoClientImpl; import org.apache.tajo.client.TajoClientUtil; @@ -41,7 +40,7 @@ import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanTestRuleProvider; import org.apache.tajo.master.TajoMaster; -import org.apache.tajo.master.rm.TajoWorkerResourceManager; +import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.plan.rewrite.LogicalPlanTestRuleProvider; import org.apache.tajo.querymaster.Query; import org.apache.tajo.querymaster.QueryMasterTask; @@ -57,7 +56,6 @@ import java.io.IOException; import java.io.Writer; import java.net.InetSocketAddress; -import java.net.URISyntaxException; import java.sql.ResultSet; import java.util.ArrayList; import java.util.List; @@ -130,7 +128,7 @@ void initPropertiesAndConfigs() { // default resource manager if (System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname) != null) { String testResourceManager = System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname); - Preconditions.checkState(testResourceManager.equals(TajoWorkerResourceManager.class.getCanonicalName())); + Preconditions.checkState(testResourceManager.equals(TajoResourceManager.class.getCanonicalName())); conf.set(ConfVars.RESOURCE_MANAGER_CLASS.varname, System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname)); } conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 2048); diff --git a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java index 2c997a3203..e5262d8d69 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java @@ -49,15 +49,15 @@ public class TestTajoResourceManager { int workerMemoryMB = 512 * 10; WorkerResourceAllocationResponse response; - private TajoWorkerResourceManager initResourceManager() throws Exception { + private TajoResourceManager initResourceManager() throws Exception { tajoConf = new org.apache.tajo.conf.TajoConf(); tajoConf.setFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT, 0.0f); tajoConf.setIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB, 512); tajoConf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS, "localhost:0"); - TajoWorkerResourceManager tajoWorkerResourceManager = new TajoWorkerResourceManager(tajoConf); - tajoWorkerResourceManager.init(tajoConf); - tajoWorkerResourceManager.start(); + TajoResourceManager tajoResourceManager = new TajoResourceManager(tajoConf); + tajoResourceManager.init(tajoConf); + tajoResourceManager.start(); for(int i = 0; i < numWorkers; i++) { ServerStatusProto.System system = ServerStatusProto.System.newBuilder() @@ -100,36 +100,36 @@ private TajoWorkerResourceManager initResourceManager() throws Exception { .setServerStatus(serverStatus) .build(); - tajoWorkerResourceManager.getResourceTracker().heartbeat(null, tajoHeartbeat, NullCallback.get()); + tajoResourceManager.getResourceTracker().heartbeat(null, tajoHeartbeat, NullCallback.get()); } - return tajoWorkerResourceManager; + return tajoResourceManager; } @Test public void testHeartbeat() throws Exception { - TajoWorkerResourceManager tajoWorkerResourceManager = null; + TajoResourceManager tajoResourceManager = null; try { - tajoWorkerResourceManager = initResourceManager(); - assertEquals(numWorkers, tajoWorkerResourceManager.getWorkers().size()); - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + tajoResourceManager = initResourceManager(); + assertEquals(numWorkers, tajoResourceManager.getWorkers().size()); + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); } } finally { - if (tajoWorkerResourceManager != null) { - tajoWorkerResourceManager.stop(); + if (tajoResourceManager != null) { + tajoResourceManager.stop(); } } } @Test public void testMemoryResource() throws Exception { - TajoWorkerResourceManager tajoWorkerResourceManager = null; + TajoResourceManager tajoResourceManager = null; try { - tajoWorkerResourceManager = initResourceManager(); + tajoResourceManager = initResourceManager(); final int minMemory = 256; final int maxMemory = 512; @@ -160,7 +160,7 @@ public void run(WorkerResourceAllocationResponse response) { } }; - tajoWorkerResourceManager.allocateWorkerResources(request, callBack); + tajoResourceManager.allocateWorkerResources(request, callBack); assertTrue(barrier.await(3, TimeUnit.SECONDS)); @@ -168,7 +168,7 @@ public void run(WorkerResourceAllocationResponse response) { int totalUsedMemory = 0; int totalUsedDisks = 0; - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); assertEquals(0, resource.getAvailableMemoryMB()); assertEquals(0, resource.getAvailableDiskSlots(), 0); @@ -190,10 +190,10 @@ public void run(WorkerResourceAllocationResponse response) { } for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) { - tajoWorkerResourceManager.releaseWorkerResource(eachContainerId); + tajoResourceManager.releaseWorkerResource(eachContainerId); } - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); assertEquals(0, resource.getUsedMemoryMB()); @@ -202,18 +202,18 @@ public void run(WorkerResourceAllocationResponse response) { assertEquals(0.0f, resource.getUsedDiskSlots(), 0); } } finally { - if (tajoWorkerResourceManager != null) { - tajoWorkerResourceManager.stop(); + if (tajoResourceManager != null) { + tajoResourceManager.stop(); } } } @Test public void testMemoryNotCommensurable() throws Exception { - TajoWorkerResourceManager tajoWorkerResourceManager = null; + TajoResourceManager tajoResourceManager = null; try { - tajoWorkerResourceManager = initResourceManager(); + tajoResourceManager = initResourceManager(); final int minMemory = 200; final int maxMemory = 500; @@ -247,7 +247,7 @@ public void run(WorkerResourceAllocationResponse response) { } }; - tajoWorkerResourceManager.allocateWorkerResources(request, callBack); + tajoResourceManager.allocateWorkerResources(request, callBack); assertTrue(barrier.await(3, TimeUnit.SECONDS)); @@ -258,10 +258,10 @@ public void run(WorkerResourceAllocationResponse response) { TestTajoResourceManager.this.response.getWorkerAllocatedResourceList()) { assertTrue( eachResource.getAllocatedMemoryMB() >= minMemory && eachResource.getAllocatedMemoryMB() <= maxMemory); - tajoWorkerResourceManager.releaseWorkerResource(eachResource.getContainerId()); + tajoResourceManager.releaseWorkerResource(eachResource.getContainerId()); } - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); assertEquals(0, resource.getUsedMemoryMB()); assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); @@ -278,7 +278,7 @@ public void run(WorkerResourceAllocationResponse response) { } } - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); assertEquals(0, resource.getUsedMemoryMB()); assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); @@ -287,18 +287,18 @@ public void run(WorkerResourceAllocationResponse response) { assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); } } finally { - if (tajoWorkerResourceManager != null) { - tajoWorkerResourceManager.stop(); + if (tajoResourceManager != null) { + tajoResourceManager.stop(); } } } @Test public void testDiskResource() throws Exception { - TajoWorkerResourceManager tajoWorkerResourceManager = null; + TajoResourceManager tajoResourceManager = null; try { - tajoWorkerResourceManager = initResourceManager(); + tajoResourceManager = initResourceManager(); final float minDiskSlots = 1.0f; final float maxDiskSlots = 2.0f; @@ -330,7 +330,7 @@ public void run(WorkerResourceAllocationResponse response) { } }; - tajoWorkerResourceManager.allocateWorkerResources(request, callBack); + tajoResourceManager.allocateWorkerResources(request, callBack); assertTrue(barrier.await(3, TimeUnit.SECONDS)); for(WorkerAllocatedResource eachResource: response.getWorkerAllocatedResourceList()) { @@ -342,7 +342,7 @@ public void run(WorkerResourceAllocationResponse response) { // assert after callback int totalUsedDisks = 0; - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); //each worker allocated 3 container (2 disk slot = 2, 1 disk slot = 1) assertEquals(0, resource.getAvailableDiskSlots(), 0); @@ -357,10 +357,10 @@ public void run(WorkerResourceAllocationResponse response) { assertEquals(numWorkers * 3, response.getWorkerAllocatedResourceList().size()); for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) { - tajoWorkerResourceManager.releaseWorkerResource(eachContainerId); + tajoResourceManager.releaseWorkerResource(eachContainerId); } - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); assertEquals(0, resource.getUsedMemoryMB()); @@ -369,18 +369,18 @@ public void run(WorkerResourceAllocationResponse response) { assertEquals(0.0f, resource.getUsedDiskSlots(), 0); } } finally { - if (tajoWorkerResourceManager != null) { - tajoWorkerResourceManager.stop(); + if (tajoResourceManager != null) { + tajoResourceManager.stop(); } } } @Test public void testDiskResourceWithStoppedQuery() throws Exception { - TajoWorkerResourceManager tajoWorkerResourceManager = null; + TajoResourceManager tajoResourceManager = null; try { - tajoWorkerResourceManager = initResourceManager(); + tajoResourceManager = initResourceManager(); final float minDiskSlots = 1.0f; final float maxDiskSlots = 2.0f; @@ -412,15 +412,15 @@ public void run(WorkerResourceAllocationResponse response) { } }; - tajoWorkerResourceManager.getRMContext().getStoppedQueryIds().add(queryId); - tajoWorkerResourceManager.allocateWorkerResources(request, callBack); + tajoResourceManager.getRMContext().getStoppedQueryIds().add(queryId); + tajoResourceManager.allocateWorkerResources(request, callBack); assertFalse(barrier.await(3, TimeUnit.SECONDS)); assertNull(response); // assert after callback int totalUsedDisks = 0; - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); //each worker allocated 3 container (2 disk slot = 2, 1 disk slot = 1) assertEquals(5.0f, resource.getAvailableDiskSlots(), 0); @@ -433,10 +433,10 @@ public void run(WorkerResourceAllocationResponse response) { assertEquals(0, totalUsedDisks, 0); for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) { - tajoWorkerResourceManager.releaseWorkerResource(eachContainerId); + tajoResourceManager.releaseWorkerResource(eachContainerId); } - for(Worker worker: tajoWorkerResourceManager.getWorkers().values()) { + for(Worker worker: tajoResourceManager.getWorkers().values()) { WorkerResource resource = worker.getResource(); assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); assertEquals(0, resource.getUsedMemoryMB()); @@ -445,8 +445,8 @@ public void run(WorkerResourceAllocationResponse response) { assertEquals(0.0f, resource.getUsedDiskSlots(), 0); } } finally { - if (tajoWorkerResourceManager != null) { - tajoWorkerResourceManager.stop(); + if (tajoResourceManager != null) { + tajoResourceManager.stop(); } } } From 1bfa430fd65006b79bfcdc382b0b7aeb17bd9686 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 16 Jun 2015 20:13:30 +0900 Subject: [PATCH 03/80] TAJO-1397 --- .../dictionary/ClusterTableDescriptor.java | 6 +- .../java/org/apache/tajo/conf/TajoConf.java | 1 - .../tajo/master/QueryCoordinatorService.java | 44 +- .../apache/tajo/master/QueryInProgress.java | 72 +- .../org/apache/tajo/master/QueryManager.java | 39 +- .../tajo/master/TajoContainerProxy.java | 42 +- .../org/apache/tajo/master/TajoMaster.java | 3 - .../tajo/master/TajoMasterClientService.java | 42 +- .../tajo/master/event/TaskRequestEvent.java | 31 +- .../NonForwardQueryResultSystemScanner.java | 27 +- .../tajo/master/rm/TajoResourceManager.java | 467 +--------- .../tajo/master/rm/TajoResourceTracker.java | 122 ++- .../org/apache/tajo/master/rm/Worker.java | 65 +- .../tajo/master/rm/WorkerResourceManager.java | 14 +- .../tajo/master/rm/WorkerStatusEvent.java | 8 +- .../master/scheduler/QuerySchedulingInfo.java | 30 +- .../scheduler/SchedulingAlgorithms.java | 4 +- .../master/scheduler/SimpleFifoScheduler.java | 148 --- .../querymaster/DefaultTaskScheduler.java | 224 ++++- .../apache/tajo/querymaster/QueryMaster.java | 32 +- .../QueryMasterManagerService.java | 27 +- .../tajo/querymaster/QueryMasterTask.java | 54 +- .../org/apache/tajo/querymaster/Stage.java | 87 +- .../resource/DefaultResourceCalculator.java | 4 +- .../java/org/apache/tajo/util/JSPUtil.java | 18 +- .../tajo/util/history/HistoryWriter.java | 22 +- .../tajo/worker/ExecutionBlockContext.java | 30 +- .../apache/tajo/worker/LegacyTaskImpl.java | 844 ------------------ .../tajo/worker/NodeResourceManager.java | 29 +- .../apache/tajo/worker/NodeStatusUpdater.java | 45 +- .../tajo/worker/TajoResourceAllocator.java | 288 +++--- .../org/apache/tajo/worker/TajoWorker.java | 90 +- .../tajo/worker/TajoWorkerManagerService.java | 54 +- .../org/apache/tajo/worker/TaskExecutor.java | 15 +- .../org/apache/tajo/worker/TaskManager.java | 20 +- .../org/apache/tajo/worker/TaskRunner.java | 306 ------- .../apache/tajo/worker/TaskRunnerManager.java | 238 ----- .../tajo/worker/WorkerHeartbeatService.java | 262 ------ .../event/ExecutionBlockStartEvent.java | 6 +- .../event/NodeResourceAllocateEvent.java | 2 +- .../tajo/worker/event/NodeResourceEvent.java | 4 +- .../tajo/worker/event/TaskRunnerEvent.java | 42 - .../worker/event/TaskRunnerStartEvent.java | 39 - .../worker/event/TaskRunnerStopEvent.java | 29 - .../tajo/ws/rs/responses/WorkerResponse.java | 44 +- .../main/proto/QueryCoordinatorProtocol.proto | 72 +- .../src/main/proto/QueryMasterProtocol.proto | 2 + .../main/proto/ResourceTrackerProtocol.proto | 6 +- .../src/main/proto/TajoWorkerProtocol.proto | 27 +- .../resources/webapps/worker/querytasks.jsp | 13 +- .../org/apache/tajo/TajoTestingCluster.java | 13 +- .../master/rm/TestTajoResourceManager.java | 66 +- .../master/scheduler/TestFifoScheduler.java | 152 ++-- .../tajo/querymaster/TestKillQuery.java | 133 +-- .../tajo/worker/MockExecutionBlock.java | 4 +- .../tajo/worker/MockNodeResourceManager.java | 6 +- .../tajo/worker/MockNodeStatusUpdater.java | 11 +- .../apache/tajo/worker/MockTaskExecutor.java | 4 +- .../apache/tajo/worker/MockTaskManager.java | 6 +- .../apache/tajo/worker/MockWorkerContext.java | 20 - .../org/apache/tajo/worker/TestFetcher.java | 14 +- .../org/apache/tajo/worker/TestHistory.java | 7 +- .../tajo/worker/TestNodeResourceManager.java | 30 +- .../tajo/worker/TestNodeStatusUpdater.java | 61 +- .../apache/tajo/worker/TestTaskExecutor.java | 41 +- .../apache/tajo/worker/TestTaskManager.java | 37 +- 66 files changed, 1209 insertions(+), 3536 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ClusterTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ClusterTableDescriptor.java index e3c830f9db..69067f424b 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ClusterTableDescriptor.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ClusterTableDescriptor.java @@ -31,10 +31,8 @@ class ClusterTableDescriptor extends AbstractTableDescriptor { new ColumnDescriptor("total_cpu", Type.INT4, 0), new ColumnDescriptor("used_mem", Type.INT8, 0), new ColumnDescriptor("total_mem", Type.INT8, 0), - new ColumnDescriptor("free_heap", Type.INT8, 0), - new ColumnDescriptor("max_heap", Type.INT8, 0), - new ColumnDescriptor("used_diskslots", Type.FLOAT4, 0), - new ColumnDescriptor("total_diskslots", Type.FLOAT4, 0), + new ColumnDescriptor("used_disk", Type.INT4, 0), + new ColumnDescriptor("total_disk", Type.INT4, 0), new ColumnDescriptor("running_tasks", Type.INT4, 0), new ColumnDescriptor("last_heartbeat_ts", Type.TIMESTAMP, 0) }; 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 90212f220e..1aebba7c23 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 @@ -156,7 +156,6 @@ public static enum ConfVars implements ConfigKey { // QueryMaster resource TAJO_QUERYMASTER_MEMORY_MB("tajo.qm.resource.memory-mb", 512, Validators.min("64")), - TAJO_QUERYMASTER_ALLOCATION_TIMEOUT("tajo.qm.resource.allocation.timeout", "3 sec"), // Tajo Worker Service Addresses WORKER_INFO_ADDRESS("tajo.worker.info-http.address", "0.0.0.0:28080", Validators.networkAddr()), diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java index 1b1d49e499..330451dac2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java @@ -23,13 +23,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.ContainerProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.rm.Worker; -import org.apache.tajo.master.rm.WorkerResource; +import org.apache.tajo.master.scheduler.event.ResourceReserveSchedulerEvent; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto; @@ -37,7 +37,6 @@ import java.net.InetSocketAddress; import java.util.Collection; -import java.util.List; public class QueryCoordinatorService extends AbstractService { private final static Log LOG = LogFactory.getLog(QueryCoordinatorService.class); @@ -113,46 +112,25 @@ public void heartbeat( builder.setResponseCommand(command); } - builder.setClusterResourceSummary(context.getResourceManager().getClusterResourceSummary()); done.run(builder.build()); } @Override - public void allocateWorkerResources( - RpcController controller, - QueryCoordinatorProtocol.WorkerResourceAllocationRequest request, - RpcCallback done) { - context.getResourceManager().allocateWorkerResources(request, done); + public void reserveNodeResources(RpcController controller, NodeResourceRequestProto request, + RpcCallback done) { + Dispatcher dispatcher = context.getResourceManager().getRMContext().getDispatcher(); + dispatcher.getEventHandler().handle(new ResourceReserveSchedulerEvent(request, done)); } @Override - public void releaseWorkerResource(RpcController controller, WorkerResourceReleaseRequest request, - RpcCallback done) { - List containerIds = request.getContainerIdsList(); + public void getAllWorkers(RpcController controller, PrimitiveProtos.NullProto request, + RpcCallback done) { - for(ContainerProtocol.TajoContainerIdProto eachContainer: containerIds) { - context.getResourceManager().releaseWorkerResource(eachContainer); - } - done.run(BOOL_TRUE); - } - - @Override - public void getAllWorkerResource(RpcController controller, PrimitiveProtos.NullProto request, - RpcCallback done) { - - WorkerResourcesRequest.Builder builder = WorkerResourcesRequest.newBuilder(); - Collection workers = context.getResourceManager().getWorkers().values(); + WorkerConnectionsProto.Builder builder = WorkerConnectionsProto.newBuilder(); + Collection workers = context.getResourceManager().getRMContext().getWorkers().values(); for(Worker worker: workers) { - WorkerResource resource = worker.getResource(); - - WorkerResourceProto.Builder workerResource = WorkerResourceProto.newBuilder(); - - workerResource.setConnectionInfo(worker.getConnectionInfo().getProto()); - workerResource.setMemoryMB(resource.getMemoryMB()); - workerResource.setDiskSlots(resource.getDiskSlots()); - - builder.addWorkerResources(workerResource); + builder.addWorker(worker.getConnectionInfo().getProto()); } done.run(builder.build()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java index f0dc0042ac..1936f28b91 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java @@ -23,21 +23,21 @@ import org.apache.hadoop.util.StringUtils; import org.apache.tajo.QueryId; import org.apache.tajo.TajoProtos; -import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.WorkerAllocatedResource; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; 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.QueryExecutionRequestProto; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.rm.TajoResourceManager; -import org.apache.tajo.master.rm.WorkerResourceManager; import org.apache.tajo.plan.logical.LogicalRootNode; import org.apache.tajo.rpc.*; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.session.Session; import org.apache.tajo.util.NetUtils; +import java.net.ConnectException; import java.net.InetSocketAddress; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -54,7 +54,7 @@ public class QueryInProgress { private LogicalRootNode plan; - private AtomicBoolean querySubmitted = new AtomicBoolean(false); + private volatile boolean querySubmitted = false; private AtomicBoolean stopped = new AtomicBoolean(false); @@ -66,6 +66,9 @@ public class QueryInProgress { private QueryMasterProtocolService queryMasterRpcClient; + //FIXME + private QueryCoordinatorProtocol.AllocationResourceProto allocation; + private final Lock readLock; private final Lock writeLock; @@ -112,7 +115,7 @@ public void stopProgress() { LOG.info("========================================================="); LOG.info("Stop query:" + queryId); - masterContext.getResourceManager().releaseQueryMaster(queryId); + masterContext.getResourceManager().getScheduler().stopQuery(queryId); RpcClientManager.cleanup(queryMasterRpc); @@ -123,7 +126,7 @@ public void stopProgress() { } } - public boolean startQueryMaster() { + public boolean startQueryMaster(QueryCoordinatorProtocol.AllocationResourceProto allocation) { try { writeLock.lockInterruptibly(); } catch (Exception e) { @@ -131,20 +134,29 @@ public boolean startQueryMaster() { return false; } try { - LOG.info("Initializing QueryInProgress for QueryID=" + queryId); + this.allocation = allocation; TajoResourceManager resourceManager = masterContext.getResourceManager(); - resourceManager.getScheduler().getq - //WorkerAllocatedResource resource = resourceManager.allocateQueryMaster(this); + WorkerConnectionInfo connectionInfo = + resourceManager.getRMContext().getWorkers().get(allocation.getWorkerId()).getConnectionInfo(); + try { + if(queryMasterRpcClient == null) { + connectQueryMaster(connectionInfo); + } + + CallFuture callFuture = new CallFuture(); + queryMasterRpcClient.startQueryMaster(callFuture.getController(), allocation, callFuture); + + if(!callFuture.get().getValue()) return false; - // if no resource to allocate a query master - if(resource == null) { - throw new RuntimeException("No Available Resources for QueryMaster"); + } catch (ConnectException ce) { + return false; } - queryInfo.setQueryMaster(resource.getConnectionInfo().getHost()); - queryInfo.setQueryMasterPort(resource.getConnectionInfo().getQueryMasterPort()); - queryInfo.setQueryMasterclientPort(resource.getConnectionInfo().getClientPort()); - queryInfo.setQueryMasterInfoPort(resource.getConnectionInfo().getHttpInfoPort()); + LOG.info("Initializing QueryInProgress for QueryID=" + queryId); + queryInfo.setQueryMaster(connectionInfo.getHost()); + queryInfo.setQueryMasterPort(connectionInfo.getQueryMasterPort()); + queryInfo.setQueryMasterclientPort(connectionInfo.getClientPort()); + queryInfo.setQueryMasterInfoPort(connectionInfo.getHttpInfoPort()); return true; } catch (Exception e) { @@ -155,31 +167,29 @@ public boolean startQueryMaster() { } } - private void connectQueryMaster() throws Exception { - InetSocketAddress addr = NetUtils.createSocketAddr(queryInfo.getQueryMasterHost(), queryInfo.getQueryMasterPort()); - LOG.info("Connect to QueryMaster:" + addr); - + private void connectQueryMaster(WorkerConnectionInfo connectionInfo) + throws NoSuchMethodException, ConnectException, ClassNotFoundException { RpcClientManager.cleanup(queryMasterRpc); + + InetSocketAddress addr = NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getQueryMasterPort()); + LOG.info("Connect to QueryMaster:" + addr); queryMasterRpc = RpcClientManager.getInstance().newClient(addr, QueryMasterProtocol.class, true); queryMasterRpcClient = queryMasterRpc.getStub(); } - public void submitQueryToMaster() { - if(querySubmitted.get()) { - return; + public boolean submitQueryToMaster() { + if(querySubmitted) { + return false; } try { writeLock.lockInterruptibly(); } catch (Exception e) { LOG.error("Failed to lock by exception " + e.getMessage(), e); - return; + return false; } try { - if(queryMasterRpcClient == null) { - connectQueryMaster(); - } LOG.info("Call executeQuery to :" + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort() + "," + queryId); @@ -195,14 +205,16 @@ public void submitQueryToMaster() { queryMasterRpcClient.executeQuery(callFuture.getController(), builder.build(), callFuture); callFuture.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); - querySubmitted.set(true); + querySubmitted = true; getQueryInfo().setQueryState(TajoProtos.QueryState.QUERY_MASTER_LAUNCHED); + return true; } catch (Exception e) { LOG.error("Failed to submit query " + queryId + " to master by exception " + e, e); catchException(e.getMessage(), e); } finally { writeLock.unlock(); } + return false; } public void catchException(String message, Throwable e) { @@ -224,10 +236,6 @@ public QueryInfo getQueryInfo() { } } - public boolean isStarted() { - return !stopped.get() && this.querySubmitted.get(); - } - public void heartbeat(QueryInfo queryInfo) { LOG.info("Received QueryMaster heartbeat:" + queryInfo); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java index daba7f7227..771edf73a4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java @@ -36,17 +36,13 @@ import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.scheduler.QuerySchedulingInfo; -import org.apache.tajo.master.scheduler.SimpleFifoScheduler; import org.apache.tajo.plan.logical.LogicalRootNode; import org.apache.tajo.querymaster.QueryJobEvent; import org.apache.tajo.session.Session; import org.apache.tajo.util.history.HistoryReader; import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.atomic.AtomicLong; /** @@ -61,8 +57,6 @@ public class QueryManager extends CompositeService { private AsyncDispatcher dispatcher; - private SimpleFifoScheduler scheduler; - private final Map submittedQueries = Maps.newConcurrentMap(); private final Map runningQueries = Maps.newConcurrentMap(); @@ -86,7 +80,6 @@ public void serviceInit(Configuration conf) throws Exception { this.dispatcher.register(QueryJobEvent.Type.class, new QueryJobManagerEventHandler()); - this.scheduler = new SimpleFifoScheduler(this); } catch (Exception e) { LOG.error("Failed to init service " + getName() + " by exception " + e, e); } @@ -101,13 +94,12 @@ public void serviceStop() throws Exception { eachQueryInProgress.stopProgress(); } } - this.scheduler.stop(); + super.serviceStop(); } @Override public void serviceStart() throws Exception { - this.scheduler.start(); super.serviceStart(); } @@ -172,6 +164,9 @@ public QueryInfo createNewSimpleQuery(QueryContext queryContext, Session session return queryInProgress.getQueryInfo(); } + /** + * submit query to scheduler + */ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, String sql, String jsonExpr, LogicalRootNode plan) throws Exception { @@ -182,28 +177,26 @@ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, Strin queryInProgress.getQueryInfo().setQueryMaster(""); submittedQueries.put(queryInProgress.getQueryId(), queryInProgress); - //scheduler.addQuery(queryInProgress); - QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo(queryInProgress.getQueryId(), 1, - queryInProgress.getQueryInfo().getStartTime()); + QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo("default", queryContext.getUser(), + queryInProgress.getQueryId(), 1, queryInProgress.getQueryInfo().getStartTime()); + masterContext.getResourceManager().submitQuery(querySchedulingInfo); return queryInProgress.getQueryInfo(); } - public QueryInfo startQueryJob(QueryId queryId) throws Exception { - - QueryInProgress queryInProgress; - - queryInProgress = submittedQueries.remove(queryId); - runningQueries.put(queryInProgress.getQueryId(), queryInProgress); + public boolean startQueryJob(QueryId queryId, QueryCoordinatorProtocol.AllocationResourceProto allocation) { - if (queryInProgress.startQueryMaster()) { + if (submittedQueries.get(queryId).startQueryMaster(allocation)) { + QueryInProgress queryInProgress = submittedQueries.remove(queryId); + runningQueries.put(queryInProgress.getQueryId(), queryInProgress); dispatcher.getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, queryInProgress.getQueryInfo())); } else { - masterContext.getQueryJobManager().stopQuery(queryInProgress.getQueryId()); + //masterContext.getQueryJobManager().stopQuery(queryInProgress.getQueryId()); + return false; } - return queryInProgress.getQueryInfo(); + return true; } class QueryJobManagerEventHandler implements EventHandler { @@ -221,7 +214,7 @@ public void handle(QueryJobEvent event) { queryInProgress.submitQueryToMaster(); } else if (event.getType() == QueryJobEvent.Type.QUERY_JOB_KILL) { - scheduler.removeQuery(queryInProgress.getQueryId()); + queryInProgress.kill(); stopQuery(queryInProgress.getQueryId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java index 2aac00561b..d66674dbc2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java @@ -61,7 +61,7 @@ public TajoContainerProxy(QueryMasterTask.QueryMasterTaskContext context, @Override public synchronized void launch(ContainerLaunchContext containerLaunchContext) { - context.getResourceAllocator().addContainer(containerId, this); + //context.getResourceAllocator().addContainer(containerId, this); this.hostName = container.getNodeId().getHost(); this.port = ((TajoWorkerContainer)container).getWorkerResource().getConnectionInfo().getPullServerPort(); @@ -104,19 +104,19 @@ private void assignExecutionBlock(ExecutionBlockId executionBlockId, TajoContain PlanProto.ShuffleType shuffleType = context.getQuery().getStage(executionBlockId).getDataChannel().getShuffleType(); - TajoWorkerProtocol.RunExecutionBlockRequestProto request = - TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder() - .setExecutionBlockId(executionBlockId.getProto()) - .setQueryMaster(context.getQueryMasterContext().getWorkerContext().getConnectionInfo().getProto()) - .setNodeId(container.getNodeId().toString()) - .setContainerId(container.getId().toString()) - .setQueryOutputPath(context.getStagingDir().toString()) - .setQueryContext(queryContext.getProto()) - .setPlanJson(planJson) - .setShuffleType(shuffleType) - .build(); - - tajoWorkerRpcClient.startExecutionBlock(null, request, NullCallback.get()); +// TajoWorkerProtocol.RunExecutionBlockRequestProto request = +// TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder() +// .setExecutionBlockId(executionBlockId.getProto()) +// .setQueryMaster(context.getQueryMasterContext().getWorkerContext().getConnectionInfo().getProto()) +// .setNodeId(container.getNodeId().toString()) +// .setContainerId(container.getId().toString()) +// .setQueryOutputPath(context.getStagingDir().toString()) +// .setQueryContext(queryContext.getProto()) +// .setPlanJson(planJson) +// .setShuffleType(shuffleType) +// .build(); +// +// tajoWorkerRpcClient.startExecutionBlock(null, request, NullCallback.get()); } catch (Throwable e) { LOG.error(e.getMessage(), e); } @@ -136,7 +136,7 @@ public synchronized void stopContainer() { } else { try { releaseWorkerResource(context, executionBlockId, Arrays.asList(containerId)); - context.getResourceAllocator().removeContainer(containerId); + //context.getResourceAllocator().removeContainer(containerId); } catch (Throwable t) { // ignore the cleanup failure String message = "cleanup failed for container " @@ -166,12 +166,12 @@ public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext tmClient = manager.getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); QueryCoordinatorProtocol.QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); - masterClientService.releaseWorkerResource(null, - QueryCoordinatorProtocol.WorkerResourceReleaseRequest.newBuilder() - .setExecutionBlockId(executionBlockId.getProto()) - .addAllContainerIds(containerIdProtos) - .build(), - NullCallback.get()); +// masterClientService.releaseWorkerResource(null, +// QueryCoordinatorProtocol.WorkerResourceReleaseRequest.newBuilder() +// .setExecutionBlockId(executionBlockId.getProto()) +// .addAllContainerIds(containerIdProtos) +// .build(), +// NullCallback.get()); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 1c731890c2..e9062cdfa5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -229,9 +229,6 @@ private void initSystemMetrics() { } private void initResourceManager() throws Exception { - Class resourceManagerClass = (Class) - systemConf.getClass(ConfVars.RESOURCE_MANAGER_CLASS.varname, TajoResourceManager.class); - Constructor constructor = resourceManagerClass.getConstructor(MasterContext.class); resourceManager = new TajoResourceManager(context); addIfService(resourceManager); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java index 4fcdc888e0..37428a7cab 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java @@ -648,27 +648,27 @@ public GetClusterInfoResponse getClusterInfo(RpcController controller, WorkerResourceInfo.Builder workerBuilder = WorkerResourceInfo.newBuilder(); - - for(Worker worker: workers.values()) { - WorkerResource workerResource = worker.getResource(); - - workerBuilder.setConnectionInfo(worker.getConnectionInfo().getProto()); - workerBuilder.setDiskSlots(workerResource.getDiskSlots()); - workerBuilder.setCpuCoreSlots(workerResource.getCpuCoreSlots()); - workerBuilder.setMemoryMB(workerResource.getMemoryMB()); - workerBuilder.setLastHeartbeat(worker.getLastHeartbeatTime()); - workerBuilder.setUsedMemoryMB(workerResource.getUsedMemoryMB()); - workerBuilder.setUsedCpuCoreSlots(workerResource.getUsedCpuCoreSlots()); - workerBuilder.setUsedDiskSlots(workerResource.getUsedDiskSlots()); - workerBuilder.setWorkerStatus(worker.getState().toString()); - workerBuilder.setMaxHeap(workerResource.getMaxHeap()); - workerBuilder.setFreeHeap(workerResource.getFreeHeap()); - workerBuilder.setTotalHeap(workerResource.getTotalHeap()); - workerBuilder.setNumRunningTasks(workerResource.getNumRunningTasks()); - workerBuilder.setNumQueryMasterTasks(workerResource.getNumQueryMasterTasks()); - - builder.addWorkerList(workerBuilder.build()); - } + //FIXME +// for(Worker worker: workers.values()) { +// WorkerResource workerResource = worker.getAvailableResource(); +// +// workerBuilder.setConnectionInfo(worker.getConnectionInfo().getProto()); +// workerBuilder.setDiskSlots(workerResource.getDiskSlots()); +// workerBuilder.setCpuCoreSlots(workerResource.getCpuCoreSlots()); +// workerBuilder.setMemoryMB(workerResource.getMemoryMB()); +// workerBuilder.setLastHeartbeat(worker.getLastHeartbeatTime()); +// workerBuilder.setUsedMemoryMB(workerResource.getUsedMemoryMB()); +// workerBuilder.setUsedCpuCoreSlots(workerResource.getUsedCpuCoreSlots()); +// workerBuilder.setUsedDiskSlots(workerResource.getUsedDiskSlots()); +// workerBuilder.setWorkerStatus(worker.getState().toString()); +// workerBuilder.setMaxHeap(workerResource.getMaxHeap()); +// workerBuilder.setFreeHeap(workerResource.getFreeHeap()); +// workerBuilder.setTotalHeap(workerResource.getTotalHeap()); +// workerBuilder.setNumRunningTasks(workerResource.getNumRunningTasks()); +// workerBuilder.setNumQueryMasterTasks(workerResource.getNumQueryMasterTasks()); +// +// builder.addWorkerList(workerBuilder.build()); +// } return builder.build(); } catch (Throwable t) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java index 3f72ed9d1e..16ceccc2ce 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java @@ -21,6 +21,7 @@ import com.google.protobuf.RpcCallback; import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType; @@ -32,36 +33,28 @@ public enum TaskRequestEventType { TASK_REQ } - private final int workerId; - private final TajoContainerId containerId; + private final QueryCoordinatorProtocol.AllocationResourceProto responseProto; private final ExecutionBlockId executionBlockId; - - private final RpcCallback callback; + private final int workerId; public TaskRequestEvent(int workerId, - TajoContainerId containerId, - ExecutionBlockId executionBlockId, - RpcCallback callback) { + QueryCoordinatorProtocol.AllocationResourceProto responseProto, + ExecutionBlockId executionBlockId) { super(TaskRequestEventType.TASK_REQ); this.workerId = workerId; - this.containerId = containerId; + this.responseProto = responseProto; this.executionBlockId = executionBlockId; - this.callback = callback; - } - - public int getWorkerId() { - return this.workerId; - } - - public TajoContainerId getContainerId() { - return this.containerId; } public ExecutionBlockId getExecutionBlockId() { return executionBlockId; } - public RpcCallback getCallback() { - return this.callback; + public QueryCoordinatorProtocol.AllocationResourceProto getResponseProto() { + return responseProto; + } + + public int getWorkerId() { + return workerId; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java index bc0d212ca0..6a82f294d8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java @@ -50,6 +50,7 @@ import org.apache.tajo.plan.logical.IndexScanNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.ScanNode; +import org.apache.tajo.resource.NodeResource; import org.apache.tajo.session.InvalidSessionException; import org.apache.tajo.storage.RowStoreUtil; import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder; @@ -423,7 +424,7 @@ private List getAllPartitions(Schema outSchema) { private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { List columns = outSchema.getRootColumns(); Tuple aTuple = new VTuple(outSchema.size()); - WorkerResource aResource = aWorker.getResource(); + NodeResource aResource = aWorker.getAvailableResource(); for (int fieldId = 0; fieldId < columns.size(); fieldId++) { Column column = columns.get(fieldId); @@ -446,11 +447,7 @@ private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { aTuple.put(fieldId, DatumFactory.createText(aWorker.getState().toString())); } else if ("RUNNING".equalsIgnoreCase(aWorker.getState().toString())) { if ("running_tasks".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aResource.getNumQueryMasterTasks())); - } else if ("free_heap".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aResource.getFreeHeap())); - } else if ("max_heap".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aResource.getMaxHeap())); + aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getNumRunningQueryMaster())); } else if ("last_heartbeat_ts".equalsIgnoreCase(column.getSimpleName())) { if (aWorker.getLastHeartbeatTime() > 0) { aTuple.put(fieldId, DatumFactory.createTimestmpDatumWithJavaMillis(aWorker.getLastHeartbeatTime())); @@ -469,7 +466,7 @@ private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) { List columns = outSchema.getRootColumns(); Tuple aTuple = new VTuple(outSchema.size()); - WorkerResource aResource = aWorker.getResource(); + NodeResource aResource = aWorker.getAvailableResource(); for (int fieldId = 0; fieldId < columns.size(); fieldId++) { Column column = columns.get(fieldId); @@ -492,21 +489,17 @@ private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) { aTuple.put(fieldId, DatumFactory.createText(aWorker.getState().toString())); } else if ("RUNNING".equalsIgnoreCase(aWorker.getState().toString())) { if ("total_cpu".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aResource.getCpuCoreSlots())); + aTuple.put(fieldId, DatumFactory.createInt4(aResource.getVirtualCores())); } else if ("used_mem".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aResource.getUsedMemoryMB()*1048576l)); + aTuple.put(fieldId, DatumFactory.createInt8(aResource.getMemory() * 1048576l)); } else if ("total_mem".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aResource.getMemoryMB()*1048576l)); - } else if ("free_heap".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aResource.getFreeHeap())); - } else if ("max_heap".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aResource.getMaxHeap())); + aTuple.put(fieldId, DatumFactory.createInt8(aWorker.getTotalResourceCapability().getMemory() * 1048576l)); } else if ("used_diskslots".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createFloat4(aResource.getUsedDiskSlots())); + aTuple.put(fieldId, DatumFactory.createInt4(aResource.getDisks())); } else if ("total_diskslots".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createFloat4(aResource.getDiskSlots())); + aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getTotalResourceCapability().getDisks())); } else if ("running_tasks".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aResource.getNumRunningTasks())); + aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getNumRunningTasks())); } else if ("last_heartbeat_ts".equalsIgnoreCase(column.getSimpleName())) { if (aWorker.getLastHeartbeatTime() > 0) { aTuple.put(fieldId, DatumFactory.createTimestmpDatumWithJavaMillis(aWorker.getLastHeartbeatTime())); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java index 1d4c85bb6a..b8c3042652 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java @@ -19,42 +19,24 @@ package org.apache.tajo.master.rm; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import com.google.protobuf.RpcCallback; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; -import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tajo.QueryId; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; -import org.apache.tajo.master.QueryInProgress; -import org.apache.tajo.master.QueryInfo; import org.apache.tajo.master.TajoMaster; import org.apache.tajo.master.scheduler.QuerySchedulingInfo; import org.apache.tajo.master.scheduler.SimpleScheduler; import org.apache.tajo.master.scheduler.TajoScheduler; -import org.apache.tajo.resource.NodeResources; -import org.apache.tajo.rpc.CancelableRpcCallback; -import org.apache.tajo.rpc.RpcUtils; -import org.apache.tajo.util.ApplicationIdUtils; -import org.apache.tajo.util.BasicFuture; +import org.apache.tajo.master.scheduler.event.SchedulerEventType; import org.apache.tajo.util.TUtil; import java.io.IOException; import java.util.*; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - /** * It manages all resources of tajo workers. @@ -63,34 +45,20 @@ public class TajoResourceManager extends CompositeService { /** class logger */ private static final Log LOG = LogFactory.getLog(TajoResourceManager.class); - static AtomicInteger containerIdSeq = new AtomicInteger(0); - private TajoMaster.MasterContext masterContext; private TajoRMContext rmContext; private String queryIdSeed; - private WorkerResourceAllocationThread workerResourceAllocator; - /** * Worker Liveliness monitor */ private WorkerLivelinessMonitor workerLivelinessMonitor; - private final BlockingQueue requestQueue = - new LinkedBlockingDeque(); - private final RpcUtils.Scrutineer> summaryRequest = - new RpcUtils.Scrutineer>(); - - private AtomicBoolean stopped = new AtomicBoolean(false); - private TajoConf systemConf; private TajoScheduler scheduler; - private ConcurrentMap allocatedResourceMap = Maps - .newConcurrentMap(); - /** It receives status messages from workers and their resources. */ private TajoResourceTracker resourceTracker; @@ -114,8 +82,6 @@ public void serviceInit(Configuration conf) throws Exception { this.queryIdSeed = String.valueOf(System.currentTimeMillis()); - workerResourceAllocator = new WorkerResourceAllocationThread(); - workerResourceAllocator.start(); this.workerLivelinessMonitor = new WorkerLivelinessMonitor(this.rmContext.getDispatcher()); addIfService(this.workerLivelinessMonitor); @@ -127,8 +93,10 @@ public void serviceInit(Configuration conf) throws Exception { addIfService(resourceTracker); //TODO configuable - scheduler = new SimpleScheduler(rmContext); + scheduler = new SimpleScheduler(masterContext); addIfService(scheduler); + rmContext.getDispatcher().register(SchedulerEventType.class, scheduler); + super.serviceInit(systemConf); } @@ -170,14 +138,6 @@ public Collection getQueryMasters() { @Override public void serviceStop() throws Exception { - if(stopped.getAndSet(true)) { - return; - } - - if(workerResourceAllocator != null) { - workerResourceAllocator.interrupt(); - } - super.serviceStop(); } @@ -198,423 +158,8 @@ public TajoScheduler getScheduler() { return scheduler; } - private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceRequest(QueryInfo queryInfo) { - int queryMasterDefaultMemoryMB = masterContext.getConf().getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB); - - QueryCoordinatorProtocol.NodeResourceRequestProto.Builder builder = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); - builder.setQueryId(queryInfo.getQueryId().getProto()) - .setAllocation(NodeResources.createResource(queryMasterDefaultMemoryMB).getProto()) - .setType(QueryCoordinatorProtocol.ResourceType.QUERYMASTER) - .setPriority(1) - .setNumContainers(1) - .setUserId(queryInfo.getQueryContext().getUser()); - //TODO .setQueue(queryInfo.getQueue()); - return builder.build(); - } - - /** - * Submit a query to scheduler - * - * @param queryInProgress QueryInProgress - * @return boolean - */ - public boolean submitQuery(QuerySchedulingInfo schedulingInfo) { - - // 3 seconds, by default - long timeout = masterContext.getConf().getTimeVar( - TajoConf.ConfVars.TAJO_QUERYMASTER_ALLOCATION_TIMEOUT, TimeUnit.MILLISECONDS); - - // Create a resource request for a query master - - List allocation = - scheduler.submitQuery(schedulingInfo.getQueryId(), createQMResourceRequest(queryInProgress.getQueryInfo())); - - - // call future for async call - final CancelableRpcCallback callFuture = - new CancelableRpcCallback() { - @Override - protected void cancel(WorkerResourceAllocationResponse canceled) { - if (canceled != null && !canceled.getWorkerAllocatedResourceList().isEmpty()) { - LOG.info("Canceling resources allocated"); - WorkerAllocatedResource resource = canceled.getWorkerAllocatedResource(0); - releaseWorkerResource(resource.getContainerId()); - } - } - }; - allocateWorkerResources(qmResourceRequest, callFuture); - - WorkerResourceAllocationResponse response = null; - try { - response = callFuture.get(timeout, TimeUnit.MILLISECONDS); - } catch (Throwable t) { - response = callFuture.cancel(); // try cancel - if (response == null) { - // canceled successfuly - LOG.warn("Got exception waiting resources for query master " + queryInProgress.getQueryId(), t); - return null; - } - } - - if (response == null || response.getWorkerAllocatedResourceList().size() == 0) { - return null; - } - - WorkerAllocatedResource resource = response.getWorkerAllocatedResource(0); - registerQueryMaster(queryInProgress.getQueryId(), resource.getContainerId()); - return resource; - } - - private void registerQueryMaster(QueryId queryId, ContainerProtocol.TajoContainerIdProto containerId) { - rmContext.getQueryMasterContainer().putIfAbsent(queryId, containerId); - } - - @Override - public void allocateWorkerResources(WorkerResourceAllocationRequest request, - RpcCallback callBack) { - try { - //TODO checking queue size - requestQueue.put(new WorkerResourceRequest(new QueryId(request.getQueryId()), false, request, callBack)); - } catch (InterruptedException e) { - LOG.error(e.getMessage(), e); - } - } - - static class WorkerResourceRequest { - boolean queryMasterRequest; - QueryId queryId; - WorkerResourceAllocationRequest request; - RpcCallback callBack; - WorkerResourceRequest( - QueryId queryId, - boolean queryMasterRequest, WorkerResourceAllocationRequest request, - RpcCallback callBack) { - this.queryId = queryId; - this.queryMasterRequest = queryMasterRequest; - this.request = request; - this.callBack = callBack; - } - } - - static class AllocatedWorkerResource { - Worker worker; - int allocatedMemoryMB; - float allocatedDiskSlots; - } - - private static final long QUEUE_POLLING_TIME = 100; - - class WorkerResourceAllocationThread extends Thread { - @Override - public void run() { - LOG.info("WorkerResourceAllocationThread start"); - while(!stopped.get()) { - BasicFuture future = summaryRequest.expire(); - if (future != null) { - future.done(makeClusterResourceSummary()); - } - try { - WorkerResourceRequest resourceRequest = requestQueue.poll( - QUEUE_POLLING_TIME, TimeUnit.MILLISECONDS); - if (resourceRequest == null) { - continue; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("allocateWorkerResources:" + - (new QueryId(resourceRequest.request.getQueryId())) + - ", requiredMemory:" + resourceRequest.request.getMinMemoryMBPerContainer() + - "~" + resourceRequest.request.getMaxMemoryMBPerContainer() + - ", requiredContainers:" + resourceRequest.request.getNumContainers() + - ", requiredDiskSlots:" + resourceRequest.request.getMinDiskSlotPerContainer() + - "~" + resourceRequest.request.getMaxDiskSlotPerContainer() + - ", queryMasterRequest=" + resourceRequest.queryMasterRequest + - ", liveWorkers=" + rmContext.getWorkers().size()); - } - - // TajoResourceManager can't return allocated disk slots occasionally. - // Because the rest resource request can remains after QueryMaster stops. - // Thus we need to find whether QueryId stopped or not. - if (!rmContext.getStoppedQueryIds().contains(resourceRequest.queryId)) { - List allocatedWorkerResources = chooseWorkers(resourceRequest); - - if(allocatedWorkerResources.size() > 0) { - List allocatedResources = - new ArrayList(); - - for(AllocatedWorkerResource allocatedResource: allocatedWorkerResources) { - NodeId nodeId = NodeId.newInstance(allocatedResource.worker.getConnectionInfo().getHost(), - allocatedResource.worker.getConnectionInfo().getPeerRpcPort()); - - TajoWorkerContainerId containerId = new TajoWorkerContainerId(); - - containerId.setApplicationAttemptId( - ApplicationIdUtils.createApplicationAttemptId(resourceRequest.queryId)); - containerId.setId(containerIdSeq.incrementAndGet()); - - ContainerProtocol.TajoContainerIdProto containerIdProto = containerId.getProto(); - allocatedResources.add(WorkerAllocatedResource.newBuilder() - .setContainerId(containerIdProto) - .setConnectionInfo(allocatedResource.worker.getConnectionInfo().getProto()) - .setAllocatedMemoryMB(allocatedResource.allocatedMemoryMB) - .setAllocatedDiskSlots(allocatedResource.allocatedDiskSlots) - .build()); - - - allocatedResourceMap.putIfAbsent(containerIdProto, allocatedResource); - } - - resourceRequest.callBack.run(WorkerResourceAllocationResponse.newBuilder() - .setQueryId(resourceRequest.request.getQueryId()) - .addAllWorkerAllocatedResource(allocatedResources) - .build() - ); - - } else { - if(LOG.isDebugEnabled()) { - LOG.debug("========================================="); - LOG.debug("Available Workers"); - for(Worker worker: rmContext.getWorkers().values()) { - LOG.debug(worker.toString()); - } - LOG.debug("========================================="); - } - requestQueue.put(resourceRequest); - Thread.sleep(QUEUE_POLLING_TIME); - } - } - } catch(InterruptedException ie) { - LOG.error(ie); - } catch (Throwable t) { - LOG.error(t, t); - } - } - } - } - - private static final long MAX_WAIT_TIME = 10000; - - public ClusterResourceSummary getClusterResourceSummary() { - BasicFuture future = - summaryRequest.check(new BasicFuture()); - try { - return future.get(MAX_WAIT_TIME, TimeUnit.MILLISECONDS); - } catch (Exception e) { - LOG.warn("Failed to get cluster summary by exception", e); - } - return null; - } - - private ClusterResourceSummary makeClusterResourceSummary() { - - int totalDiskSlots = 0; - int totalCpuCoreSlots = 0; - int totalMemoryMB = 0; - - int totalAvailableDiskSlots = 0; - int totalAvailableCpuCoreSlots = 0; - int totalAvailableMemoryMB = 0; - - for(Worker worker: rmContext.getWorkers().values()) { - - WorkerResource resource = worker.getResource(); - - totalMemoryMB += resource.getMemoryMB(); - totalAvailableMemoryMB += resource.getAvailableMemoryMB(); - - totalDiskSlots += resource.getDiskSlots(); - totalAvailableDiskSlots += resource.getAvailableDiskSlots(); - - totalCpuCoreSlots += resource.getCpuCoreSlots(); - totalAvailableCpuCoreSlots += resource.getAvailableCpuCoreSlots(); - } - - return ClusterResourceSummary.newBuilder() - .setNumWorkers(rmContext.getWorkers().size()) - .setTotalCpuCoreSlots(totalCpuCoreSlots) - .setTotalDiskSlots(totalDiskSlots) - .setTotalMemoryMB(totalMemoryMB) - .setTotalAvailableCpuCoreSlots(totalAvailableCpuCoreSlots) - .setTotalAvailableDiskSlots(totalAvailableDiskSlots) - .setTotalAvailableMemoryMB(totalAvailableMemoryMB) - .build(); - } - - private List chooseWorkers(WorkerResourceRequest resourceRequest) { - List selectedWorkers = new ArrayList(); - - int allocatedResources = 0; - - ResourceRequestPriority resourceRequestPriority - = resourceRequest.request.getResourceRequestPriority(); - - List randomWorkers = new ArrayList(rmContext.getWorkers().values()); - Collections.shuffle(randomWorkers); - - if(resourceRequestPriority == ResourceRequestPriority.MEMORY) { - - int numContainers = resourceRequest.request.getNumContainers(); - int minMemoryMB = resourceRequest.request.getMinMemoryMBPerContainer(); - int maxMemoryMB = resourceRequest.request.getMaxMemoryMBPerContainer(); - float diskSlot = Math.max(resourceRequest.request.getMaxDiskSlotPerContainer(), - resourceRequest.request.getMinDiskSlotPerContainer()); - - int liveWorkerSize = randomWorkers.size(); - Set insufficientWorkers = new HashSet(); - boolean stop = false; - boolean checkMax = true; - while(!stop) { - if(allocatedResources >= numContainers) { - break; - } - - if(insufficientWorkers.size() >= liveWorkerSize) { - if(!checkMax) { - break; - } - insufficientWorkers.clear(); - checkMax = false; - } - int compareAvailableMemory = checkMax ? maxMemoryMB : minMemoryMB; - - for(Worker worker: randomWorkers) { - if(allocatedResources >= numContainers) { - stop = true; - break; - } - - if(insufficientWorkers.size() >= liveWorkerSize) { - break; - } - - WorkerResource workerResource = worker.getResource(); - if(workerResource.getAvailableMemoryMB() >= compareAvailableMemory) { - int workerMemory; - if(workerResource.getAvailableMemoryMB() >= maxMemoryMB) { - workerMemory = maxMemoryMB; - } else { - workerMemory = workerResource.getAvailableMemoryMB(); - } - AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource(); - allocatedWorkerResource.worker = worker; - allocatedWorkerResource.allocatedMemoryMB = workerMemory; - if(workerResource.getAvailableDiskSlots() >= diskSlot) { - allocatedWorkerResource.allocatedDiskSlots = diskSlot; - } else { - allocatedWorkerResource.allocatedDiskSlots = workerResource.getAvailableDiskSlots(); - } - - workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots, - allocatedWorkerResource.allocatedMemoryMB); - - selectedWorkers.add(allocatedWorkerResource); - - allocatedResources++; - } else { - insufficientWorkers.add(worker.getWorkerId()); - } - } - } - } else { - int numContainers = resourceRequest.request.getNumContainers(); - float minDiskSlots = resourceRequest.request.getMinDiskSlotPerContainer(); - float maxDiskSlots = resourceRequest.request.getMaxDiskSlotPerContainer(); - int memoryMB = Math.max(resourceRequest.request.getMaxMemoryMBPerContainer(), - resourceRequest.request.getMinMemoryMBPerContainer()); - - int liveWorkerSize = randomWorkers.size(); - Set insufficientWorkers = new HashSet(); - boolean stop = false; - boolean checkMax = true; - while(!stop) { - if(allocatedResources >= numContainers) { - break; - } - - if(insufficientWorkers.size() >= liveWorkerSize) { - if(!checkMax) { - break; - } - insufficientWorkers.clear(); - checkMax = false; - } - float compareAvailableDisk = checkMax ? maxDiskSlots : minDiskSlots; - - for(Worker worker: randomWorkers) { - if(allocatedResources >= numContainers) { - stop = true; - break; - } - - if(insufficientWorkers.size() >= liveWorkerSize) { - break; - } - - WorkerResource workerResource = worker.getResource(); - if(workerResource.getAvailableDiskSlots() >= compareAvailableDisk) { - float workerDiskSlots; - if(workerResource.getAvailableDiskSlots() >= maxDiskSlots) { - workerDiskSlots = maxDiskSlots; - } else { - workerDiskSlots = workerResource.getAvailableDiskSlots(); - } - AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource(); - allocatedWorkerResource.worker = worker; - allocatedWorkerResource.allocatedDiskSlots = workerDiskSlots; - - if(workerResource.getAvailableMemoryMB() >= memoryMB) { - allocatedWorkerResource.allocatedMemoryMB = memoryMB; - } else { - allocatedWorkerResource.allocatedMemoryMB = workerResource.getAvailableMemoryMB(); - } - workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots, - allocatedWorkerResource.allocatedMemoryMB); - - selectedWorkers.add(allocatedWorkerResource); - - allocatedResources++; - } else { - insufficientWorkers.add(worker.getWorkerId()); - } - } - } - } - return selectedWorkers; - } - - /** - * Release allocated resource. - * - * @param containerId ContainerIdProto to be released - */ - @Override - public void releaseWorkerResource(ContainerProtocol.TajoContainerIdProto containerId) { - AllocatedWorkerResource allocated = allocatedResourceMap.remove(containerId); - if(allocated != null) { - LOG.info("Release Resource: " + allocated.allocatedDiskSlots + "," + allocated.allocatedMemoryMB); - allocated.worker.getResource().releaseResource( allocated.allocatedDiskSlots, allocated.allocatedMemoryMB); - } else { - LOG.warn("No AllocatedWorkerResource data for [" + containerId + "]"); - return; - } - } - - @Override - public boolean isQueryMasterStopped(QueryId queryId) { - return !rmContext.getQueryMasterContainer().containsKey(queryId); - } - - @Override - public void releaseQueryMaster(QueryId queryId) { - if(!rmContext.getQueryMasterContainer().containsKey(queryId)) { - LOG.warn("No QueryMaster resource info for " + queryId); - return; - } else { - ContainerProtocol.TajoContainerIdProto containerId = rmContext.getQueryMasterContainer().remove(queryId); - releaseWorkerResource(containerId); - rmContext.getStoppedQueryIds().add(queryId); - LOG.info(String.format("Released QueryMaster (%s) resource." , queryId.toString())); - } + public void submitQuery(QuerySchedulingInfo schedulingInfo) { + scheduler.submitQuery(schedulingInfo); } public TajoRMContext getRMContext() { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index acdcd0ee14..7402913485 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -25,14 +25,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeatResponse; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.master.scheduler.event.SchedulerEvent; +import org.apache.tajo.master.scheduler.event.SchedulerEventType; import org.apache.tajo.resource.NodeResource; -import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.util.NetUtils; -import org.apache.tajo.util.ProtoUtil; import java.io.IOError; import java.net.InetSocketAddress; @@ -112,25 +111,22 @@ public void serviceStop() throws Exception { super.serviceStop(); } - /** The response builder */ - private static final TajoHeartbeatResponse.Builder builder = - TajoHeartbeatResponse.newBuilder().setHeartbeatResult(ProtoUtil.TRUE); - - private static WorkerStatusEvent createStatusEvent(int workerId, NodeHeartbeatRequestProto heartbeat) { + private static WorkerStatusEvent createStatusEvent(NodeHeartbeatRequestProto heartbeat) { return new WorkerStatusEvent( - workerId, - heartbeat.ggetServerStatus().getRunningTaskNum(), - heartbeat.getServerStatus().getJvmHeap().getMaxHeap(), - heartbeat.getServerStatus().getJvmHeap().getFreeHeap(), - heartbeat.getServerStatus().getJvmHeap().getTotalHeap()); + heartbeat.getWorkerId(), + heartbeat.getRunningTasks(), + heartbeat.getRunningQueryMasters(), + new NodeResource(heartbeat.getAvailableResource())); } @Override - public void heartbeat( + public void nodeHeartbeat( RpcController controller, NodeHeartbeatRequestProto heartbeat, RpcCallback done) { + NodeHeartbeatResponseProto.Builder response = NodeHeartbeatResponseProto.newBuilder(); + ResponseCommand responseCommand = ResponseCommand.NORMAL; try { // get a workerId from the heartbeat int workerId = heartbeat.getWorkerId(); @@ -140,78 +136,68 @@ public void heartbeat( if (heartbeat.hasAvailableResource()) { // status update rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(heartbeat)); + rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); } // refresh ping workerLivelinessMonitor.receivedPing(workerId); } else if (rmContext.getInactiveWorkers().containsKey(workerId)) { // worker was inactive + if (!heartbeat.hasConnectionInfo()) { + // request membership to worker node + responseCommand = ResponseCommand.MEMBERSHIP; + } else { - // remove the inactive worker from the list of inactive workers. - Worker worker = rmContext.getInactiveWorkers().remove(workerId); - workerLivelinessMonitor.unregister(worker.getWorkerId()); + // remove the inactive worker from the list of inactive workers. + Worker worker = rmContext.getInactiveWorkers().remove(workerId); + workerLivelinessMonitor.unregister(worker.getWorkerId()); - // create new worker instance - Worker newWorker = createWorkerResource(heartbeat); - int newWorkerId = newWorker.getWorkerId(); - // add the new worker to the list of active workers - rmContext.getWorkers().putIfAbsent(newWorkerId, newWorker); + // create new worker instance + Worker newWorker = createWorkerResource(heartbeat); + int newWorkerId = newWorker.getWorkerId(); + // add the new worker to the list of active workers + rmContext.getWorkers().putIfAbsent(newWorkerId, newWorker); - // Transit the worker to RUNNING - rmContext.getDispatcher().getEventHandler().handle(new WorkerEvent(newWorkerId, WorkerEventType.STARTED)); - // register the worker to the liveliness monitor - workerLivelinessMonitor.register(newWorkerId); + // Transit the worker to RUNNING + rmContext.getDispatcher().getEventHandler().handle(new WorkerEvent(newWorkerId, WorkerEventType.STARTED)); + // register the worker to the liveliness monitor + workerLivelinessMonitor.register(newWorkerId); - } else { // if new worker pings firstly + rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); + } - // create new worker instance - Worker newWorker = createWorkerResource(heartbeat); - Worker oldWorker = rmContext.getWorkers().putIfAbsent(workerId, newWorker); + } else { // if new worker pings firstly - if (oldWorker == null) { - // Transit the worker to RUNNING - rmContext.rmDispatcher.getEventHandler().handle(new WorkerEvent(workerId, WorkerEventType.STARTED)); + // The pings have not membership information + if (!heartbeat.hasConnectionInfo()) { + // request membership to worker node + responseCommand = ResponseCommand.MEMBERSHIP; } else { - LOG.info("Reconnect from the node at: " + workerId); - workerLivelinessMonitor.unregister(workerId); - rmContext.getDispatcher().getEventHandler().handle(new WorkerReconnectEvent(workerId, newWorker)); - } - workerLivelinessMonitor.register(workerId); + // create new worker instance + Worker newWorker = createWorkerResource(heartbeat); + Worker oldWorker = rmContext.getWorkers().putIfAbsent(workerId, newWorker); + + if (oldWorker == null) { + // Transit the worker to RUNNING + rmContext.rmDispatcher.getEventHandler().handle(new WorkerEvent(workerId, WorkerEventType.STARTED)); + } else { + LOG.info("Reconnect from the node at: " + workerId); + workerLivelinessMonitor.unregister(workerId); + rmContext.getDispatcher().getEventHandler().handle(new WorkerReconnectEvent(workerId, newWorker)); + } + + workerLivelinessMonitor.register(workerId); + rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); + } } - } finally { - builder.setClusterResourceSummary(manager.getClusterResourceSummary()); - done.run(builder.build()); + done.run(response.setCommand(responseCommand).build()); } } - @Override - public void nodeHeartbeat(RpcController controller, NodeHeartbeatRequestProto request, - RpcCallback done) { - //TODO implement with ResourceManager for scheduler - TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.Builder - response = TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.newBuilder(); - done.run(response.setCommand(TajoResourceTrackerProtocol.ResponseCommand.NORMAL).build()); - } - - private Worker createWorkerResource(NodeHeartbeat request) { - WorkerResource workerResource = new WorkerResource(); - - if(request.getServerStatus() != null) { - workerResource.setMemoryMB(request.getServerStatus().getMemoryResourceMB()); - workerResource.setCpuCoreSlots(request.getServerStatus().getSystem().getAvailableProcessors()); - workerResource.setDiskSlots(request.getServerStatus().getDiskSlots()); - workerResource.setNumRunningTasks(request.getServerStatus().getRunningTaskNum()); - workerResource.setMaxHeap(request.getServerStatus().getJvmHeap().getMaxHeap()); - workerResource.setFreeHeap(request.getServerStatus().getJvmHeap().getFreeHeap()); - workerResource.setTotalHeap(request.getServerStatus().getJvmHeap().getTotalHeap()); - } else { - workerResource.setMemoryMB(4096); - workerResource.setDiskSlots(4); - workerResource.setCpuCoreSlots(4); - } - - return new Worker(rmContext, workerResource, new WorkerConnectionInfo(request.getConnectionInfo())); + private Worker createWorkerResource(NodeHeartbeatRequestProto request) { + return new Worker(rmContext, new NodeResource(request.getTotalResource()), + new WorkerConnectionInfo(request.getConnectionInfo())); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java index 0a38a191b2..5494c00e6f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java @@ -57,17 +57,20 @@ public class Worker implements EventHandler, Comparable { static { HEARTBEAT_TIME_UPDATER = PlatformDependent.newAtomicLongFieldUpdater(Worker.class, "lastHeartbeatTime"); if (HEARTBEAT_TIME_UPDATER == null) { - HEARTBEAT_TIME_UPDATER = AtomicLongFieldUpdater.newUpdater(NodeResource.class, "lastHeartbeatTime"); - RUNNING_TASK_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeResource.class, "numRunningTasks"); - RUNNING_QM_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeResource.class, "numRunningQueryMaster"); + HEARTBEAT_TIME_UPDATER = AtomicLongFieldUpdater.newUpdater(Worker.class, "lastHeartbeatTime"); + RUNNING_TASK_UPDATER = AtomicIntegerFieldUpdater.newUpdater(Worker.class, "numRunningTasks"); + RUNNING_QM_UPDATER = AtomicIntegerFieldUpdater.newUpdater(Worker.class, "numRunningQueryMaster"); } else { - RUNNING_TASK_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeResource.class, "numRunningTasks"); - RUNNING_QM_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeResource.class, "numRunningQueryMaster"); + RUNNING_TASK_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(Worker.class, "numRunningTasks"); + RUNNING_QM_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(Worker.class, "numRunningQueryMaster"); } } - /** Resource capability */ - private final NodeResource resource; + /** Available resources on the node. */ + private final NodeResource availableResource; + + /** Total resources on the node. */ + private final NodeResource totalResourceCapability; /** Worker connection information */ private WorkerConnectionInfo connectionInfo; @@ -114,12 +117,13 @@ public class Worker implements EventHandler, Comparable { private final StateMachine stateMachine = stateMachineFactory.make(this, WorkerState.NEW); - public Worker(TajoRMContext rmContext, NodeResource resource, WorkerConnectionInfo connectionInfo) { + public Worker(TajoRMContext rmContext, NodeResource totalResourceCapability, WorkerConnectionInfo connectionInfo) { this.rmContext = rmContext; this.connectionInfo = connectionInfo; this.lastHeartbeatTime = System.currentTimeMillis(); - this.resource = resource; + this.totalResourceCapability = totalResourceCapability; + this.availableResource = NodeResources.clone(totalResourceCapability); } public int getWorkerId() { @@ -138,10 +142,18 @@ public void setNumRunningQueryMaster(int numRunningQueryMaster) { RUNNING_QM_UPDATER.lazySet(this, numRunningQueryMaster); } + public int getNumRunningQueryMaster() { + return numRunningQueryMaster; + } + public void setNumRunningTasks(int numRunningTasks) { RUNNING_TASK_UPDATER.lazySet(this, numRunningTasks); } + public int getNumRunningTasks() { + return numRunningTasks; + } + public long getLastHeartbeatTime() { return this.lastHeartbeatTime; } @@ -155,11 +167,21 @@ public WorkerState getState() { } /** + * Get current resources on the node. + * + * @return current resources on the node. + */ + public NodeResource getAvailableResource() { + return this.availableResource; + } + + /** + * Get total resources on the node. * - * @return the current resource capability of worker + * @return total resources on the node. */ - public NodeResource getResource() { - return this.resource; + public NodeResource getTotalResourceCapability() { + return totalResourceCapability; } @Override @@ -177,25 +199,15 @@ public boolean equals(Object o) { Worker worker = (Worker) o; - if (lastHeartbeatTime != worker.lastHeartbeatTime) return false; if (connectionInfo != null ? !connectionInfo.equals(worker.connectionInfo) : worker.connectionInfo != null) return false; - - if (resource != null ? !resource.equals(worker.resource) : worker.resource != null) return false; - if (rmContext != null ? !rmContext.equals(worker.rmContext) : worker.rmContext != null) return false; - if (stateMachine != null ? !stateMachine.equals(worker.stateMachine) : worker.stateMachine != null) return false; - return true; } @Override public int hashCode() { int result = 0; - result = 31 * result + (rmContext != null ? rmContext.hashCode() : 0); - result = 31 * result + (int) (lastHeartbeatTime ^ (lastHeartbeatTime >>> 32)); - result = 31 * result + (resource != null ? resource.hashCode() : 0); result = 31 * result + (connectionInfo != null ? connectionInfo.hashCode() : 0); - result = 31 * result + (stateMachine != null ? stateMachine.hashCode() : 0); return result; } @@ -204,7 +216,7 @@ public static class AddNodeTransition implements SingleArcTransition { @@ -248,11 +260,8 @@ public static class ReconnectNodeTransition implements SingleArcTransition rpcCallBack); +// public void allocateWorkerResources(QueryCoordinatorProtocol.NodeResourceRequestProto request, +// RpcCallback rpcCallBack); /** * Release a container @@ -98,7 +94,7 @@ public void allocateWorkerResources(QueryCoordinatorProtocol.NodeResourceRequest * * @return The overall summary of cluster resources */ - public ClusterResourceSummary getClusterResourceSummary(); + //public ClusterResourceSummary getClusterResourceSummary(); /** * diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java index c117aecff2..862b4a1be7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java @@ -27,13 +27,13 @@ public class WorkerStatusEvent extends WorkerEvent { private final int runningTaskNum; private final int runningQMNum; - private final NodeResource resource; + private final NodeResource available; - public WorkerStatusEvent(int workerId, int runningTaskNum, int runningQMNum, NodeResource resource) { + public WorkerStatusEvent(int workerId, int runningTaskNum, int runningQMNum, NodeResource available) { super(workerId, WorkerEventType.STATE_UPDATE); this.runningTaskNum = runningTaskNum; this.runningQMNum = runningQMNum; - this.resource = resource; + this.available = available; } public int getRunningTaskNum() { @@ -45,6 +45,6 @@ public int getRunningQMNum() { } public NodeResource getResource() { - return resource; + return available; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java index 3dd33892ed..f88bc330d4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java @@ -18,15 +18,18 @@ package org.apache.tajo.master.scheduler; -import com.google.common.base.Objects; import org.apache.tajo.QueryId; -public class QuerySchedulingInfo { +public class QuerySchedulingInfo implements Comparable { + private String queue; + private String user; private QueryId queryId; - private Integer priority; - private Long startTime; + private int priority; + private long startTime; - public QuerySchedulingInfo(QueryId queryId, Integer priority, Long startTime) { + public QuerySchedulingInfo(String queue, String user, QueryId queryId, int priority, long startTime) { + this.queue = queue; + this.user = user; this.queryId = queryId; this.priority = priority; this.startTime = startTime; @@ -36,11 +39,11 @@ public QueryId getQueryId() { return queryId; } - public Integer getPriority() { + public int getPriority() { return priority; } - public Long getStartTime() { + public long getStartTime() { return startTime; } @@ -48,8 +51,17 @@ public String getName() { return queryId.getId(); } + public String getQueue() { + return queue; + } + + @Override - public int hashCode() { - return Objects.hashCode(startTime, getName(), priority); + public int compareTo(QuerySchedulingInfo o) { + int ret = Integer.compare(priority, o.priority); + if(ret == 0) { + ret = Long.compare(startTime, o.startTime); + } + return ret; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java index 7fd07b5544..365897e150 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java @@ -32,9 +32,9 @@ public class SchedulingAlgorithms { public static class FifoComparator implements Comparator { @Override public int compare(QuerySchedulingInfo q1, QuerySchedulingInfo q2) { - int res = q1.getPriority().compareTo(q2.getPriority()); + int res = Integer.compare(q1.getPriority(), q2.getPriority()); if (res == 0) { - res = (int) Math.signum(q1.getStartTime() - q2.getStartTime()); + res = Long.compare(q1.getStartTime(), q2.getStartTime()); } if (res == 0) { // In the rare case where jobs were submitted at the exact same time, diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.java deleted file mode 100644 index 6cb98ebc79..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.java +++ /dev/null @@ -1,148 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.master.scheduler; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.tajo.QueryId; -import org.apache.tajo.master.QueryInProgress; -import org.apache.tajo.master.QueryManager; - -import java.util.*; -import java.util.concurrent.atomic.AtomicBoolean; - -public class SimpleFifoScheduler implements Scheduler { - private static final Log LOG = LogFactory.getLog(SimpleFifoScheduler.class.getName()); - private LinkedList pool = new LinkedList(); - private final Thread queryProcessor; - private AtomicBoolean stopped = new AtomicBoolean(); - private QueryManager manager; - private Comparator COMPARATOR = new SchedulingAlgorithms.FifoComparator(); - - public SimpleFifoScheduler(QueryManager manager) { - this.manager = manager; - this.queryProcessor = new Thread(new QueryProcessor()); - this.queryProcessor.setName("Query Processor"); - } - - @Override - public Mode getMode() { - return Mode.FIFO; - } - - @Override - public String getName() { - return manager.getName(); - } - - @Override - public boolean addQuery(QueryInProgress queryInProgress) { - int qSize = pool.size(); - if (qSize != 0 && qSize % 100 == 0) { - LOG.info("Size of Fifo queue is " + qSize); - } - - QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo(queryInProgress.getQueryId(), 1, - queryInProgress.getQueryInfo().getStartTime()); - boolean result = pool.add(querySchedulingInfo); - if (getRunningQueries().size() == 0) wakeupProcessor(); - return result; - } - - @Override - public boolean removeQuery(QueryId queryId) { - return pool.remove(getQueryByQueryId(queryId)); - } - - public QuerySchedulingInfo getQueryByQueryId(QueryId queryId) { - for (QuerySchedulingInfo querySchedulingInfo : pool) { - if (querySchedulingInfo.getQueryId().equals(queryId)) { - return querySchedulingInfo; - } - } - return null; - } - - @Override - public List getRunningQueries() { - return new ArrayList(manager.getRunningQueries()); - } - - public void start() { - queryProcessor.start(); - } - - public void stop() { - if (stopped.getAndSet(true)) { - return; - } - pool.clear(); - synchronized (queryProcessor) { - queryProcessor.interrupt(); - } - } - - private QuerySchedulingInfo pollScheduledQuery() { - if (pool.size() > 1) { - Collections.sort(pool, COMPARATOR); - } - return pool.poll(); - } - - private void wakeupProcessor() { - synchronized (queryProcessor) { - queryProcessor.notifyAll(); - } - } - - private final class QueryProcessor implements Runnable { - @Override - public void run() { - - QuerySchedulingInfo query; - - while (!stopped.get() && !Thread.currentThread().isInterrupted()) { - query = null; - if (getRunningQueries().size() == 0) { - query = pollScheduledQuery(); - } - - if (query != null) { - try { - manager.startQueryJob(query.getQueryId()); - } catch (Throwable t) { - LOG.fatal("Exception during query startup:", t); - manager.stopQuery(query.getQueryId()); - } - } - - synchronized (queryProcessor) { - try { - queryProcessor.wait(500); - } catch (InterruptedException e) { - if (stopped.get()) { - break; - } - LOG.warn("Exception during shutdown: ", e); - } - } - } - } - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 939de606c2..eb5374be76 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -29,25 +29,41 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryIdFactory; import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequest; import org.apache.tajo.engine.query.TaskRequestImpl; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.ContainerProxy; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.container.TajoContainerId; +import org.apache.tajo.master.container.TajoContainerIdPBImpl; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; import org.apache.tajo.master.event.TaskSchedulerEvent.EventType; +import org.apache.tajo.master.rm.TajoWorkerContainerId; +import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.serder.LogicalNodeSerializer; import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.AsyncRpcClient; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.rpc.NettyClientBase; +import org.apache.tajo.rpc.RpcClientManager; +import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.storage.DataLocation; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.storage.fragment.Fragment; +import org.apache.tajo.util.ApplicationIdUtils; import org.apache.tajo.util.NetUtils; import org.apache.tajo.worker.FetchImpl; +import java.net.ConnectException; +import java.net.InetSocketAddress; import java.util.*; import java.util.Map.Entry; import java.util.concurrent.LinkedBlockingQueue; @@ -94,17 +110,14 @@ public void start() { public void run() { while(!stopEventHandling.get() && !Thread.currentThread().isInterrupted()) { - try { - synchronized (schedulingThread){ - schedulingThread.wait(100); - } - schedule(); - } catch (InterruptedException e) { - break; - } catch (Throwable e) { - LOG.fatal(e.getMessage(), e); - break; - } + schedule(); +// try { +// } catch (InterruptedException e) { +// break; +// } catch (Throwable e) { +// LOG.fatal(e.getMessage(), e); +// break; +// } } LOG.info("TaskScheduler schedulingThread stopped"); } @@ -145,7 +158,7 @@ public void stop() { // Return all of request callbacks instantly. if(taskRequests != null){ for (TaskRequestEvent req : taskRequests.taskRequestQueue) { - req.getCallback().run(stopTaskRunnerReq); +// req.getCallback().run(stopTaskRunnerReq); } } @@ -158,6 +171,7 @@ public void stop() { LinkedList taskRequestEvents = new LinkedList(); public void schedule() { + handleTaskRequestEvent(null); if (taskRequests.size() > 0) { if (scheduledRequests.leafTaskNum() > 0) { @@ -254,16 +268,56 @@ public void handle(TaskSchedulerEvent event) { @Override public void handleTaskRequestEvent(TaskRequestEvent event) { - taskRequests.handle(event); - int hosts = scheduledRequests.leafTaskHostMapping.size(); - // if available cluster resource are large then tasks, the scheduler thread are working immediately. - if(remainingScheduledObjectNum() > 0 && - (remainingScheduledObjectNum() <= hosts || hosts <= taskRequests.size())){ - synchronized (schedulingThread){ - schedulingThread.notifyAll(); + NettyClientBase tmClient = null; + try { + ServiceTracker serviceTracker = context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); + tmClient = RpcClientManager.getInstance(). + getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); + QueryCoordinatorProtocol.QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); + + CallFuture callBack = new CallFuture(); + QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); + request.setCapacity(NodeResources.createResource(512).getProto()); + request.setNumContainers(scheduledObjectNum); + request.setPriority(1); + request.setQueryId(context.getMasterContext().getQueryId().getProto()); + request.setQueue("default"); + request.setType(QueryCoordinatorProtocol.ResourceType.LEAF); + request.setUserId("test"); + request.setRunningTasks(0); + masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); + + QueryCoordinatorProtocol.NodeResourceResponseProto responseProto = callBack.get(); + + for (QueryCoordinatorProtocol.AllocationResourceProto proto : responseProto.getResourceList()) { + + TaskRequestEvent taskRequestEvent = new TaskRequestEvent(proto.getWorkerId(), proto, context.getBlockId()); + taskRequests.handle(taskRequestEvent); } + + if(responseProto.getResourceCount() == 0) { + synchronized (schedulingThread){ + schedulingThread.wait(50); + } + } else { + LOG.info("Allocates :" + responseProto.getResourceCount()); + } + + } catch (Throwable e) { + LOG.error(e.getMessage(), e); } + + + int hosts = scheduledRequests.leafTaskHostMapping.size(); + + // if available cluster resource are large then tasks, the scheduler thread are working immediately. +// if(remainingScheduledObjectNum() > 0 && +// (remainingScheduledObjectNum() <= hosts || hosts <= taskRequests.size())){ +// synchronized (schedulingThread){ +// schedulingThread.notifyAll(); +// } +// } } @Override @@ -278,11 +332,11 @@ private class TaskRequests implements EventHandler { @Override public void handle(TaskRequestEvent event) { if(LOG.isDebugEnabled()){ - LOG.debug("TaskRequest: " + event.getContainerId() + "," + event.getExecutionBlockId()); + LOG.debug("TaskRequest: " + event.getResponseProto().getWorkerId() + "," + event.getExecutionBlockId()); } if(stopEventHandling.get()) { - event.getCallback().run(stopTaskRunnerReq); + //event.getCallback().run(stopTaskRunnerReq); return; } int qSize = taskRequestQueue.size(); @@ -594,6 +648,17 @@ public String getRack() { } } + public void cancel(TaskAttempt taskAttempt) { + List locations = taskAttempt.getTask().getDataLocations(); + + for (DataLocation location : locations) { + HostVolumeMapping volumeMapping = scheduledRequests.leafTaskHostMapping.get(location.getHost()); + volumeMapping.addTaskAttempt(location.getVolumeId(), taskAttempt); + } + + scheduledRequests.leafTasks.add(taskAttempt.getId()); + } + private class ScheduledRequests { // two list leafTasks and nonLeafTasks keep all tasks to be scheduled. Even though some task is included in // leafTaskHostMapping or leafTasksRackMapping, some task T will not be sent to a task runner @@ -757,16 +822,13 @@ public void assignToLeafTasks(LinkedList taskRequests) { // checking if this container is still alive. // If not, ignore the task request and stop the task runner - ContainerProxy container = context.getMasterContext().getResourceAllocator() - .getContainer(taskRequest.getContainerId()); - if(container == null) { - taskRequest.getCallback().run(stopTaskRunnerReq); + WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); + if(connectionInfo == null) { + //taskRequest.getCallback().run(stopTaskRunnerReq); continue; } // getting the hostname of requested node - WorkerConnectionInfo connectionInfo = - context.getMasterContext().getResourceAllocator().getWorkerConnectionInfo(taskRequest.getWorkerId()); String host = connectionInfo.getHost(); // if there are no worker matched to the hostname a task request @@ -783,7 +845,13 @@ public void assignToLeafTasks(LinkedList taskRequests) { } } - TajoContainerId containerId = taskRequest.getContainerId(); + TajoWorkerContainerId containerId = new TajoWorkerContainerId(); + + containerId.setApplicationAttemptId( + ApplicationIdUtils.createApplicationAttemptId(taskRequest.getExecutionBlockId().getQueryId())); + containerId.setId(0); + + LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," + "containerId=" + containerId); @@ -807,7 +875,7 @@ public void assignToLeafTasks(LinkedList taskRequests) { if(hostVolumeMapping.getRemoteConcurrency() > tailLimit){ //release container hostVolumeMapping.decreaseConcurrency(containerId); - taskRequest.getCallback().run(stopTaskRunnerReq); + //taskRequest.getCallback().run(stopTaskRunnerReq); continue; } } @@ -847,18 +915,65 @@ public void assignToLeafTasks(LinkedList taskRequests) { taskAssign.setInterQuery(); } + //taskRequest.getCallback().run(taskAssign.getProto()); + + + TajoWorkerProtocol.BatchAllocationRequestProto.Builder requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); + requestProto.addTaskRequest(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + .setResource(taskRequest.getResponseProto().getResource()) + .setTaskRequest(taskAssign.getProto()).build()); + + requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); + if(!ebMap.containsKey(taskRequest.getWorkerId())) { + // first request with starting ExecutionBlock + PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); + + TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); + ebRequestProto.setExecutionBlockId(taskRequest.getExecutionBlockId().getProto()) + .setQueryMaster(connectionInfo.getProto()) + .setQueryContext(context.getMasterContext().getQueryContext().getProto()) + .setQueryOutputPath(context.getMasterContext().getStagingDir().toString()) + .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) + .setShuffleType(shuffleType); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + ebMap.put(taskRequest.getWorkerId(), true); + } + + CallFuture callFuture = new CallFuture(); + + InetSocketAddress addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); + AsyncRpcClient tajoWorkerRpc = null; + try { + tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); + TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); + tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); + TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); + + for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { + cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); + LOG.warn("cancel" + proto.getTaskRequest()); + } + + if(responseProto.getCancellationTaskCount() > 0) { + continue; + } + } catch (Exception e) { + LOG.error(e); + } + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, - taskRequest.getContainerId(), connectionInfo)); + containerId, connectionInfo)); assignedRequest.add(attemptId); scheduledObjectNum--; - taskRequest.getCallback().run(taskAssign.getProto()); + } else { throw new RuntimeException("Illegal State!!!!!!!!!!!!!!!!!!!!!"); } } } - + Map ebMap = Maps.newHashMap(); private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) { if (masterPlan.isRoot(block)) { return false; @@ -912,13 +1027,50 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { } } - WorkerConnectionInfo connectionInfo = context.getMasterContext().getResourceAllocator(). - getWorkerConnectionInfo(taskRequest.getWorkerId()); + WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, - taskRequest.getContainerId(), connectionInfo)); - taskRequest.getCallback().run(taskAssign.getProto()); + null, connectionInfo)); + //taskRequest.getCallback().run(taskAssign.getProto()); totalAssigned++; scheduledObjectNum--; + + TajoWorkerProtocol.BatchAllocationRequestProto.Builder requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); + requestProto.addTaskRequest(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + .setResource(taskRequest.getResponseProto().getResource()) + .setTaskRequest(taskAssign.getProto()).build()); + + requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); + if(!ebMap.containsKey(taskRequest.getWorkerId())) { + // first request with starting ExecutionBlock + PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); + + TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); + ebRequestProto.setExecutionBlockId(taskRequest.getExecutionBlockId().getProto()) + .setQueryMaster(connectionInfo.getProto()) + .setQueryContext(context.getMasterContext().getQueryContext().getProto()) + .setQueryOutputPath(context.getMasterContext().getStagingDir().toString()) + .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) + .setShuffleType(shuffleType); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + ebMap.put(taskRequest.getWorkerId(), true); + } + + CallFuture callFuture = new CallFuture(); + + InetSocketAddress addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); + AsyncRpcClient tajoWorkerRpc = null; + try { + tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); + TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); + tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); + TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); + LOG.info(responseProto.getCancellationTaskCount()); + } catch (Exception e) { + LOG.error(e); + } + } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index 6c5bd2227d..7e8bc43648 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -36,6 +36,7 @@ import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.QueryStartEvent; import org.apache.tajo.master.event.QueryStopEvent; import org.apache.tajo.rpc.*; @@ -167,6 +168,7 @@ public void stop() { LOG.info("QueryMaster stopped"); } + //FIXME remove this protected void cleanupExecutionBlock(List executionBlockIds) { StringBuilder cleanupMessage = new StringBuilder(); String prefix = ""; @@ -176,15 +178,14 @@ protected void cleanupExecutionBlock(List ex } LOG.info("cleanup executionBlocks: " + cleanupMessage); NettyClientBase rpc = null; - List workers = getAllWorker(); + List workers = getAllWorker(); TajoWorkerProtocol.ExecutionBlockListProto.Builder builder = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); builder.addAllExecutionBlockId(Lists.newArrayList(executionBlockIds)); TajoWorkerProtocol.ExecutionBlockListProto executionBlockListProto = builder.build(); - for (WorkerResourceProto worker : workers) { + for (TajoProtos.WorkerConnectionInfoProto worker : workers) { try { - TajoProtos.WorkerConnectionInfoProto connectionInfo = worker.getConnectionInfo(); - rpc = manager.getClient(NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()), + rpc = manager.getClient(NetUtils.createSocketAddr(worker.getHost(), worker.getPeerRpcPort()), TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); @@ -196,15 +197,15 @@ protected void cleanupExecutionBlock(List ex } } + //FIXME get workers to QueryMasterTask private void cleanup(QueryId queryId) { LOG.info("cleanup query resources : " + queryId); NettyClientBase rpc = null; - List workers = getAllWorker(); + List workers = getAllWorker(); - for (WorkerResourceProto worker : workers) { + for (TajoProtos.WorkerConnectionInfoProto worker : workers) { try { - TajoProtos.WorkerConnectionInfoProto connectionInfo = worker.getConnectionInfo(); - rpc = manager.getClient(NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()), + rpc = manager.getClient(NetUtils.createSocketAddr(worker.getHost(), worker.getPeerRpcPort()), TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); @@ -215,7 +216,7 @@ private void cleanup(QueryId queryId) { } } - public List getAllWorker() { + public List getAllWorker() { NettyClientBase rpc = null; try { @@ -228,16 +229,17 @@ public List getAllWorker() { rpc = manager.getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); QueryCoordinatorProtocolService masterService = rpc.getStub(); - CallFuture callBack = new CallFuture(); - masterService.getAllWorkerResource(callBack.getController(), + CallFuture callBack = new CallFuture(); + masterService.getAllWorkers(callBack.getController(), PrimitiveProtos.NullProto.getDefaultInstance(), callBack); - WorkerResourcesRequest workerResourcesRequest = callBack.get(2, TimeUnit.SECONDS); - return workerResourcesRequest.getWorkerResourcesList(); + WorkerConnectionsProto connectionsProto = + callBack.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); + return connectionsProto.getWorkerList(); } catch (Exception e) { LOG.error(e.getMessage(), e); } - return new ArrayList(); + return new ArrayList(); } @Override @@ -357,8 +359,10 @@ public void stopQuery(QueryId queryId) { QueryHistory queryHisory = query.getQueryHistory(); if (queryHisory != null) { try { + long startTime = System.currentTimeMillis(); query.context.getQueryMasterContext().getWorkerContext(). getTaskHistoryWriter().appendAndFlush(queryHisory); + LOG.info("QueryHistory write delay:" + (System.currentTimeMillis() - startTime)); } catch (Throwable e) { LOG.warn(e, e); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index 59933a7b60..1b15eea5a3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -28,17 +28,21 @@ import org.apache.tajo.*; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.container.TajoContainerId; import org.apache.tajo.master.event.*; +import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.session.Session; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.util.NetUtils; import org.apache.tajo.worker.TajoWorker; +import org.apache.tajo.worker.event.QMResourceAllocateEvent; import java.net.InetSocketAddress; +import java.util.concurrent.ExecutionException; public class QueryMasterManagerService extends CompositeService implements QueryMasterProtocol.QueryMasterProtocolService.Interface { @@ -125,10 +129,10 @@ public void getTask(RpcController controller, TajoWorkerProtocol.GetTaskRequestP if(queryMasterTask == null || queryMasterTask.isStopped()) { done.run(DefaultTaskScheduler.stopTaskRunnerReq); } else { - TajoContainerId cid = - queryMasterTask.getQueryTaskContext().getResourceAllocator().makeContainerId(request.getContainerId()); - LOG.debug("getTask:" + cid + ", ebId:" + ebId); - queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(request.getWorkerId(), cid, ebId, done)); +// TajoContainerId cid = +// queryMasterTask.getQueryTaskContext().getResourceAllocator().makeContainerId(request.getContainerId()); +// LOG.debug("getTask:" + cid + ", ebId:" + ebId); +// queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(request.getWorkerId(), cid, ebId, done)); } } catch (Exception e) { LOG.error(e.getMessage(), e); @@ -234,4 +238,19 @@ public void executeQuery(RpcController controller, request.getLogicalPlanJson().getValue())); done.run(TajoWorker.NULL_PROTO); } + + @Override + public void startQueryMaster(RpcController controller, + QueryCoordinatorProtocol.AllocationResourceProto request, + RpcCallback done) { + CallFuture callFuture = new CallFuture(); + workerContext.getNodeResourceManager().handle(new QMResourceAllocateEvent(request, callFuture)); + + try { + done.run(callFuture.get()); + } catch (Exception e) { + controller.setFailed(e.getMessage()); + done.run(TajoWorker.FALSE_PROTO); + } + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index 4e9554eed4..b239de942d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -18,6 +18,7 @@ package org.apache.tajo.querymaster; +import com.google.common.collect.Maps; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -42,6 +43,7 @@ import org.apache.tajo.exception.UnimplementedException; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.TajoContainerProxy; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.plan.LogicalOptimizer; @@ -54,6 +56,8 @@ import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.verifier.VerifyException; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; import org.apache.tajo.session.Session; import org.apache.tajo.storage.Tablespace; import org.apache.tajo.storage.StorageProperty; @@ -63,9 +67,12 @@ import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter; import org.apache.tajo.worker.AbstractResourceAllocator; import org.apache.tajo.worker.TajoResourceAllocator; +import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; import java.io.IOException; import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -104,7 +111,7 @@ public class QueryMasterTask extends CompositeService { private AtomicLong lastClientHeartbeat = new AtomicLong(-1); - private AbstractResourceAllocator resourceAllocator; +// private AbstractResourceAllocator resourceAllocator; private AtomicBoolean stopped = new AtomicBoolean(false); @@ -115,6 +122,8 @@ public class QueryMasterTask extends CompositeService { private final List diagnostics = new ArrayList(); + private final ConcurrentMap workerMap = Maps.newConcurrentMap(); + public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext, QueryId queryId, Session session, QueryContext queryContext, String jsonExpr, AsyncDispatcher dispatcher) { @@ -144,14 +153,7 @@ public void init(Configuration conf) { try { queryTaskContext = new QueryMasterTaskContext(); - String resourceManagerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_MANAGER_CLASS); - if(resourceManagerClassName.indexOf(TajoResourceManager.class.getName()) >= 0) { - resourceAllocator = new TajoResourceAllocator(queryTaskContext); - } else { - throw new UnimplementedException(resourceManagerClassName + " is not supported yet"); - } - addService(resourceAllocator); addService(dispatcher); dispatcher.register(StageEventType.class, new StageEventDispatcher()); @@ -179,6 +181,10 @@ public boolean isStopped() { @Override public void start() { startQuery(); + List workersProto = queryMasterContext.getQueryMaster().getAllWorker(); + for (TajoProtos.WorkerConnectionInfoProto worker : workersProto) { + workerMap.put(worker.getId(), new WorkerConnectionInfo(worker)); + } super.start(); } @@ -191,11 +197,13 @@ public void stop() { LOG.info("Stopping QueryMasterTask:" + queryId); - try { - resourceAllocator.stop(); - } catch (Throwable t) { - LOG.fatal(t.getMessage(), t); - } + getQueryTaskContext().getQueryMasterContext().getWorkerContext(). + getNodeResourceManager().getDispatcher().getEventHandler().handle(new NodeResourceDeallocateEvent(NodeResources.createResource(512).getProto())); +// try { +// resourceAllocator.stop(); +// } catch (Throwable t) { +// LOG.fatal(t.getMessage(), t); +// } if (queryMetrics != null) { queryMetrics.report(new MetricsConsoleReporter()); @@ -204,7 +212,7 @@ public void stop() { super.stop(); LOG.info("Stopped QueryMasterTask:" + queryId); } - + //FIXME remove public void handleTaskRequestEvent(TaskRequestEvent event) { ExecutionBlockId id = event.getExecutionBlockId(); query.getStage(id).handleTaskRequestEvent(event); @@ -271,10 +279,10 @@ public void handle(TaskSchedulerEvent event) { private class LocalTaskEventHandler implements EventHandler { @Override public void handle(LocalTaskEvent event) { - TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId()); - if (proxy != null) { - proxy.killTaskAttempt(event.getTaskAttemptId()); - } +// TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId()); +// if (proxy != null) { +// proxy.killTaskAttempt(event.getTaskAttemptId()); +// } } } @@ -587,12 +595,16 @@ public float getProgress() { return query.getProgress(); } - public AbstractResourceAllocator getResourceAllocator() { - return resourceAllocator; - } +// public AbstractResourceAllocator getResourceAllocator() { +// return resourceAllocator; +// } public TajoMetrics getQueryMetrics() { return queryMetrics; } + + public ConcurrentMap getWorkerMap() { + return workerMap; + } } } \ No newline at end of file diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 5a0fc38de3..8a094077a7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -20,9 +20,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.Event; @@ -52,6 +54,7 @@ import org.apache.tajo.master.TaskRunnerGroupEvent; import org.apache.tajo.master.TaskRunnerGroupEvent.EventType; import org.apache.tajo.master.TaskState; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.container.TajoContainer; import org.apache.tajo.master.container.TajoContainerId; import org.apache.tajo.master.event.*; @@ -59,6 +62,10 @@ import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.querymaster.Task.IntermediateEntry; +import org.apache.tajo.rpc.NettyClientBase; +import org.apache.tajo.rpc.NullCallback; +import org.apache.tajo.rpc.RpcClientManager; +import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.storage.FileTablespace; import org.apache.tajo.storage.Tablespace; import org.apache.tajo.storage.TableSpaceManager; @@ -70,6 +77,7 @@ import org.apache.tajo.worker.FetchImpl; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -702,9 +710,45 @@ private void stopScheduler() { } } + + //////////////FIXME + public void stopExecutionBlock(TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto) { + + for (WorkerConnectionInfo worker : getContext().getWorkerMap().values()) { + NettyClientBase tajoWorkerRpc = null; + try { + InetSocketAddress addr = new InetSocketAddress(worker.getHost(), worker.getPeerRpcPort()); + tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); + TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); + + tajoWorkerRpcClient.stopExecutionBlock(null, requestProto, NullCallback.get(PrimitiveProtos.BoolProto.class)); + } catch (Throwable e) { + LOG.error(e.getMessage(), e); + } + } + } + private void releaseContainers() { // If there are still live TaskRunners, try to kill the containers. and send the shuffle report request - eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), containers.values())); + + //eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), containers.values())); + List ebIds = Lists.newArrayList(); + if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) { + List childs = getMasterPlan().getChilds(getId()); + + for (ExecutionBlock executionBlock : childs) { + ebIds.add(executionBlock.getId().getProto()); + } + + //getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds); + } + + TajoWorkerProtocol.StopExecutionBlockRequestProto.Builder stopRequest = TajoWorkerProtocol.StopExecutionBlockRequestProto.newBuilder(); + TajoWorkerProtocol.ExecutionBlockListProto.Builder builder = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); + builder.addAllExecutionBlockId(Lists.newArrayList(ebIds)); + stopRequest.setChild(builder.build()); + stopRequest.setExecutionBlockId(getId().getProto()); + stopExecutionBlock(stopRequest.build()); } /** @@ -1048,16 +1092,21 @@ public static long getInputVolume(MasterPlan masterPlan, QueryMasterTask.QueryMa } public static void allocateContainers(Stage stage) { - ExecutionBlock execBlock = stage.getBlock(); + stage.getEventHandler().handle(new StageContainerAllocationEvent(stage.getId(), new ArrayList())); + return; + + /*ExecutionBlock execBlock = stage.getBlock(); //TODO consider disk slot int requiredMemoryMBPerTask = 512; - int numRequest = stage.getContext().getResourceAllocator().calculateNumRequestContainers( - stage.getContext().getQueryMasterContext().getWorkerContext(), - stage.schedulerContext.getEstimatedTaskNum(), - requiredMemoryMBPerTask - ); +// int numRequest = stage.getContext().getResourceAllocator().calculateNumRequestContainers( +// stage.getContext().getQueryMasterContext().getWorkerContext(), +// stage.schedulerContext.getEstimatedTaskNum(), +// requiredMemoryMBPerTask +// ); + + int numRequest = 0; final Resource resource = Records.newRecord(Resource.class); @@ -1071,7 +1120,7 @@ public static void allocateContainers(Stage stage) { new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ, stage.getId(), priority, resource, numRequest, stage.masterPlan.isLeaf(execBlock), 0.0f); - stage.eventHandler.handle(event); + stage.eventHandler.handle(event);*/ } private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException { @@ -1178,11 +1227,11 @@ public void transition(Stage stage, StageEvent event) { stage.containers.put(cId, container); } LOG.info("Stage (" + stage.getId() + ") has " + stage.containers.size() + " containers!"); - stage.eventHandler.handle( - new LaunchTaskRunnersEvent(stage.getId(), allocationEvent.getAllocatedContainer(), - stage.getContext().getQueryContext(), - CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) - ); +// stage.eventHandler.handle( +// new LaunchTaskRunnersEvent(stage.getId(), allocationEvent.getAllocatedContainer(), +// stage.getContext().getQueryContext(), +// CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) +// ); stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_START)); } catch (Throwable t) { @@ -1282,18 +1331,6 @@ public void transition(Stage stage, StageEvent stageEvent) { private void cleanup() { stopScheduler(); releaseContainers(); - - if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) { - List childs = getMasterPlan().getChilds(getId()); - List ebIds = Lists.newArrayList(); - - for (ExecutionBlock executionBlock : childs) { - ebIds.add(executionBlock.getId().getProto()); - } - - getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds); - } - this.finalStageHistory = makeStageHistory(); this.finalStageHistory.setTasks(makeTaskHistories()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java b/tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java index 58b8a26868..15ec8459e4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java +++ b/tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java @@ -27,9 +27,7 @@ public int compare(NodeResource unused, NodeResource lhs, NodeResource rhs) { @Override public int computeAvailableContainers(NodeResource available, NodeResource required) { - return Math.min(Math.min( - available.getMemory() / required.getMemory(), - available.getDisks() / required.getDisks()), + return Math.min(available.getMemory() / required.getMemory(), available.getVirtualCores() / required.getVirtualCores()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java index 578b15aabc..5021d83cf8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java @@ -19,7 +19,6 @@ package org.apache.tajo.util; import org.apache.hadoop.conf.Configuration; -import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; @@ -31,7 +30,6 @@ import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.util.history.StageHistory; import org.apache.tajo.util.history.TaskHistory; -import org.apache.tajo.worker.TaskRunner; import org.apache.tajo.worker.TaskRunnerHistory; import java.text.DecimalFormat; @@ -66,14 +64,14 @@ public static void sortTaskHistory(List tasks, String sortField, St Collections.sort(tasks, new TaskHistoryComparator(sortField, "asc".equals(sortOrder))); } - public static void sortTaskRunner(List taskRunners) { - Collections.sort(taskRunners, new Comparator() { - @Override - public int compare(TaskRunner taskRunner, TaskRunner taskRunner2) { - return taskRunner.getId().compareTo(taskRunner2.getId()); - } - }); - } +// public static void sortTaskRunner(List taskRunners) { +// Collections.sort(taskRunners, new Comparator() { +// @Override +// public int compare(TaskRunner taskRunner, TaskRunner taskRunner2) { +// return taskRunner.getId().compareTo(taskRunner2.getId()); +// } +// }); +// } public static void sortTaskRunnerHistory(List histories) { Collections.sort(histories, new Comparator() { diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java index daced3e90c..29fe79a200 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java @@ -32,6 +32,7 @@ import org.apache.tajo.master.QueryInfo; import org.apache.tajo.util.Bytes; import org.apache.tajo.worker.TaskHistory; +import org.apache.tajo.worker.event.NodeStatusEvent; import java.io.Closeable; import java.io.IOException; @@ -144,6 +145,7 @@ public void done(WriterHolder holder) { } }; historyQueue.add(future); + synchronized (writerThread) { writerThread.notifyAll(); } @@ -151,7 +153,7 @@ public void done(WriterHolder holder) { } /* synchronously flush to history file */ - public synchronized void appendAndSync(History history) + public void appendAndSync(History history) throws TimeoutException, InterruptedException, IOException { WriterFuture future = appendAndFlush(history); @@ -267,24 +269,20 @@ public void run() { private int drainHistory(Collection> buffer, int numElements, long timeoutMillis) throws InterruptedException { - long deadline = System.currentTimeMillis() + timeoutMillis; + long deadline = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeoutMillis); int added = 0; while (added < numElements) { added += historyQueue.drainTo(buffer, numElements - added); if (added < numElements) { // not enough elements immediately available; will have to wait - if (deadline <= System.currentTimeMillis()) { - break; - } else { - synchronized (writerThread) { - writerThread.wait(deadline - System.currentTimeMillis()); - if (deadline > System.currentTimeMillis()) { - added += historyQueue.drainTo(buffer, numElements - added); - break; - } - } + WriterFuture e = historyQueue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); + if (e == null) { + break; // we already waited enough, and there are no more elements in sight } + buffer.add(e); + added++; } } + return added; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 9e4a60f758..9596d3ceff 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -60,7 +60,7 @@ public class ExecutionBlockContext { /** class logger */ private static final Log LOG = LogFactory.getLog(ExecutionBlockContext.class); - private TaskRunnerManager manager; + protected AtomicInteger runningTasksNum = new AtomicInteger(); protected AtomicInteger completedTasksNum = new AtomicInteger(); protected AtomicInteger succeededTasksNum = new AtomicInteger(); protected AtomicInteger killedTasksNum = new AtomicInteger(); @@ -100,9 +100,8 @@ public class ExecutionBlockContext { private final Map taskHistories = Maps.newTreeMap(); - public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, - TaskRunnerManager manager, RunExecutionBlockRequestProto request) throws IOException { - this.manager = manager; + public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, StartExecutionBlockRequestProto request) + throws IOException { this.executionBlockId = new ExecutionBlockId(request.getExecutionBlockId()); this.connManager = RpcClientManager.getInstance(); this.queryMaster = new WorkerConnectionInfo(request.getQueryMaster()); @@ -255,21 +254,6 @@ public Task getTask(TaskAttemptId taskAttemptId){ return tasks.get(taskAttemptId); } - @Deprecated - public void stopTaskRunner(String id){ - manager.stopTaskRunner(id); - } - - @Deprecated - public TaskRunner getTaskRunner(String taskRunnerId){ - return manager.getTaskRunner(taskRunnerId); - } - - @Deprecated - public void addTaskHistory(String taskRunnerId, TaskAttemptId quAttemptId, TaskHistory taskHistory) { - histories.get(taskRunnerId).addTaskHistory(quAttemptId, taskHistory); - } - public void addTaskHistory(TaskId taskId, TaskHistory taskHistory) { taskHistories.put(taskId, taskHistory); } @@ -289,10 +273,10 @@ public void fatalError(TaskAttemptId taskAttemptId, String message) { getStub().fatalError(null, builder.build(), NullCallback.get()); } - public TaskRunnerHistory createTaskRunnerHistory(TaskRunner runner){ - histories.putIfAbsent(runner.getId(), new TaskRunnerHistory(runner.getContainerId(), executionBlockId)); - return histories.get(runner.getId()); - } +// public TaskRunnerHistory createTaskRunnerHistory(TaskRunner runner){ +// histories.putIfAbsent(runner.getId(), new TaskRunnerHistory(runner.getContainerId(), executionBlockId)); +// return histories.get(runner.getId()); +// } public TajoWorker.WorkerContext getWorkerContext(){ return workerContext; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java deleted file mode 100644 index 0721ef162e..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java +++ /dev/null @@ -1,844 +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.worker; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import io.netty.handler.codec.http.QueryStringDecoder; -import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.tajo.TajoProtos; -import org.apache.tajo.TajoProtos.TaskAttemptState; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.catalog.TableDesc; -import org.apache.tajo.catalog.TableMeta; -import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.planner.physical.PhysicalExec; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.engine.query.TaskRequest; -import org.apache.tajo.ipc.QueryMasterProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.*; -import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.function.python.TajoScriptEngine; -import org.apache.tajo.plan.logical.*; -import org.apache.tajo.plan.serder.LogicalNodeDeserializer; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.pullserver.TajoPullServerService; -import org.apache.tajo.pullserver.retriever.FileChunk; -import org.apache.tajo.rpc.NullCallback; -import org.apache.tajo.storage.*; -import org.apache.tajo.storage.fragment.FileFragment; -import org.apache.tajo.util.NetUtils; - -import java.io.File; -import java.io.IOException; -import java.net.InetAddress; -import java.net.URI; -import java.util.*; -import java.util.Map.Entry; -import java.util.concurrent.ExecutorService; - -import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; -import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; - -@Deprecated -public class LegacyTaskImpl implements Task { - private static final Log LOG = LogFactory.getLog(LegacyTaskImpl.class); - private static final float FETCHER_PROGRESS = 0.5f; - - private final TajoConf systemConf; - private final QueryContext queryContext; - private final ExecutionBlockContext executionBlockContext; - private final String taskRunnerId; - - private final Path taskDir; - private final TaskRequest request; - private TaskAttemptContext context; - private List fetcherRunners; - private LogicalNode plan; - private final Map descs = Maps.newHashMap(); - private PhysicalExec executor; - private boolean interQuery; - private Path inputTableBaseDir; - - private long startTime; - private long finishTime; - - private final TableStats inputStats; - private List localChunks; - - // TODO - to be refactored - private ShuffleType shuffleType = null; - private Schema finalSchema = null; - private TupleComparator sortComp = null; - - public LegacyTaskImpl(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { - this(taskRunnerId, baseDir, taskId, executionBlockContext.getConf(), executionBlockContext, request); - } - - public LegacyTaskImpl(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - TajoConf conf, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { - this.taskRunnerId = taskRunnerId; - this.request = request; - - this.systemConf = conf; - this.queryContext = request.getQueryContext(systemConf); - this.executionBlockContext = executionBlockContext; - this.taskDir = StorageUtil.concatPath(baseDir, - taskId.getTaskId().getId() + "_" + taskId.getId()); - - this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskId, - request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); - this.context.setDataChannel(request.getDataChannel()); - this.context.setEnforcer(request.getEnforcer()); - this.context.setState(TaskAttemptState.TA_PENDING); - this.inputStats = new TableStats(); - this.fetcherRunners = Lists.newArrayList(); - } - - public void initPlan() throws IOException { - plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); - LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); - if (scanNode != null) { - for (LogicalNode node : scanNode) { - ScanNode scan = (ScanNode) node; - descs.put(scan.getCanonicalName(), scan.getTableDesc()); - } - } - - LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); - if (partitionScanNode != null) { - for (LogicalNode node : partitionScanNode) { - PartitionedTableScanNode scan = (PartitionedTableScanNode) node; - descs.put(scan.getCanonicalName(), scan.getTableDesc()); - } - } - - interQuery = request.getProto().getInterQuery(); - if (interQuery) { - context.setInterQuery(); - this.shuffleType = context.getDataChannel().getShuffleType(); - - if (shuffleType == ShuffleType.RANGE_SHUFFLE) { - SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); - this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); - this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); - } - } else { - Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) - .getAppenderFilePath(getId(), queryContext.getStagingDir()); - LOG.info("Output File Path: " + outFilePath); - context.setOutputPath(outFilePath); - } - - this.localChunks = Collections.synchronizedList(new ArrayList()); - LOG.info("=================================="); - LOG.info("* Stage " + request.getId() + " is initialized"); - LOG.info("* InterQuery: " + interQuery - + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + - ", Fragments (num: " + request.getFragments().size() + ")" + - ", Fetches (total:" + request.getFetches().size() + ") :"); - - if(LOG.isDebugEnabled()) { - for (FetchImpl f : request.getFetches()) { - LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); - } - } - LOG.info("* Local task dir: " + taskDir); - if(LOG.isDebugEnabled()) { - LOG.debug("* plan:\n"); - LOG.debug(plan.toString()); - } - LOG.info("=================================="); - } - - private void startScriptExecutors() throws IOException { - for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.start(systemConf); - } - } - - private void stopScriptExecutors() { - for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.shutdown(); - } - } - - @Override - public void init() throws IOException { - initPlan(); - startScriptExecutors(); - - if (context.getState() == TaskAttemptState.TA_PENDING) { - // initialize a task temporal dir - FileSystem localFS = executionBlockContext.getLocalFS(); - localFS.mkdirs(taskDir); - - if (request.getFetches().size() > 0) { - inputTableBaseDir = localFS.makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( - getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); - localFS.mkdirs(inputTableBaseDir); - Path tableDir; - for (String inputTable : context.getInputTables()) { - tableDir = new Path(inputTableBaseDir, inputTable); - if (!localFS.exists(tableDir)) { - LOG.info("the directory is created " + tableDir.toUri()); - localFS.mkdirs(tableDir); - } - } - } - // for localizing the intermediate data - fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); - } - } - - private TaskAttemptId getId() { - return context.getTaskId(); - } - - public String toString() { - return "queryId: " + this.getId() + " status: " + context.getState(); - } - - @Override - public boolean isStopped() { - return context.isStopped(); - } - - @Override - public TaskAttemptContext getTaskContext() { - return context; - } - - @Override - public ExecutionBlockContext getExecutionBlockContext() { - return executionBlockContext; - } - - @Override - public boolean hasFetchPhase() { - return fetcherRunners.size() > 0; - } - - @Override - public void fetch() { - ExecutorService executorService = executionBlockContext.getTaskRunner(taskRunnerId).getFetchLauncher(); - for (Fetcher f : fetcherRunners) { - executorService.submit(new FetchRunner(context, f)); - } - } - - @Override - public void kill() { - stopScriptExecutors(); - context.setState(TaskAttemptState.TA_KILLED); - context.stop(); - } - - @Override - public void abort() { - stopScriptExecutors(); - context.setState(TajoProtos.TaskAttemptState.TA_FAILED); - context.stop(); - } - - @Override - public TaskStatusProto getReport() { - TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); - builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); - builder.setId(context.getTaskId().getProto()) - .setProgress(context.getProgress()) - .setState(context.getState()); - - builder.setInputStats(reloadInputStats()); - - if (context.getResultStats() != null) { - builder.setResultStats(context.getResultStats().getProto()); - } - return builder.build(); - } - - @Override - public boolean isProgressChanged() { - return context.isProgressChanged(); - } - - @Override - public void updateProgress() { - if(context != null && context.isStopped()){ - return; - } - - if (executor != null && context.getProgress() < 1.0f) { - context.setExecutorProgress(executor.getProgress()); - } - } - - private CatalogProtos.TableStatsProto reloadInputStats() { - synchronized(inputStats) { - if (this.executor == null) { - return inputStats.getProto(); - } - - TableStats executorInputStats = this.executor.getInputStats(); - - if (executorInputStats != null) { - inputStats.setValues(executorInputStats); - } - return inputStats.getProto(); - } - } - - private TaskCompletionReport getTaskCompletionReport() { - TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); - builder.setId(context.getTaskId().getProto()); - - builder.setInputStats(reloadInputStats()); - - if (context.hasResultStats()) { - builder.setResultStats(context.getResultStats().getProto()); - } else { - builder.setResultStats(new TableStats().getProto()); - } - - Iterator> it = context.getShuffleFileOutputs(); - if (it.hasNext()) { - do { - Entry entry = it.next(); - ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); - part.setPartId(entry.getKey()); - - // Set output volume - if (context.getPartitionOutputVolume() != null) { - for (Entry e : context.getPartitionOutputVolume().entrySet()) { - if (entry.getKey().equals(e.getKey())) { - part.setVolume(e.getValue().longValue()); - break; - } - } - } - - builder.addShuffleFileOutputs(part.build()); - } while (it.hasNext()); - } - - return builder.build(); - } - - private void waitForFetch() throws InterruptedException, IOException { - context.getFetchLatch().await(); - LOG.info(context.getTaskId() + " All fetches are done!"); - Collection inputs = Lists.newArrayList(context.getInputTables()); - - // Get all broadcasted tables - Set broadcastTableNames = new HashSet(); - List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); - if (broadcasts != null) { - for (EnforceProperty eachBroadcast : broadcasts) { - broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); - } - } - - // localize the fetched data and skip the broadcast table - for (String inputTable: inputs) { - if (broadcastTableNames.contains(inputTable)) { - continue; - } - File tableDir = new File(context.getFetchIn(), inputTable); - FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); - context.updateAssignedFragments(inputTable, frags); - } - } - - @Override - public void run() throws Exception { - startTime = System.currentTimeMillis(); - Throwable error = null; - try { - if(!context.isStopped()) { - context.setState(TaskAttemptState.TA_RUNNING); - if (context.hasFetchPhase()) { - // If the fetch is still in progress, the query unit must wait for - // complete. - waitForFetch(); - context.setFetcherProgress(FETCHER_PROGRESS); - updateProgress(); - } - - this.executor = executionBlockContext.getTQueryEngine(). - createPlan(context, plan); - this.executor.init(); - - while(!context.isStopped() && executor.next() != null) { - } - } - } catch (Throwable e) { - error = e ; - LOG.error(e.getMessage(), e); - stopScriptExecutors(); - context.stop(); - } finally { - if (executor != null) { - try { - executor.close(); - reloadInputStats(); - } catch (IOException e) { - LOG.error(e, e); - } - this.executor = null; - } - - executionBlockContext.completedTasksNum.incrementAndGet(); - context.getHashShuffleAppenderManager().finalizeTask(getId()); - - QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); - if (context.isStopped()) { - context.setExecutorProgress(0.0f); - - if (context.getState() == TaskAttemptState.TA_KILLED) { - queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); - executionBlockContext.killedTasksNum.incrementAndGet(); - } else { - context.setState(TaskAttemptState.TA_FAILED); - TaskFatalErrorReport.Builder errorBuilder = - TaskFatalErrorReport.newBuilder() - .setId(getId().getProto()); - if (error != null) { - if (error.getMessage() == null) { - errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); - } else { - errorBuilder.setErrorMessage(error.getMessage()); - } - errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); - } - - queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); - executionBlockContext.failedTasksNum.incrementAndGet(); - } - } else { - // if successful - context.stop(); - context.setProgress(1.0f); - context.setState(TaskAttemptState.TA_SUCCEEDED); - executionBlockContext.succeededTasksNum.incrementAndGet(); - - TaskCompletionReport report = getTaskCompletionReport(); - queryMasterStub.done(null, report, NullCallback.get()); - } - finishTime = System.currentTimeMillis(); - LOG.info(context.getTaskId() + " completed. " + - "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + - ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() - + ", killed: " + executionBlockContext.killedTasksNum.intValue() - + ", failed: " + executionBlockContext.failedTasksNum.intValue()); - } - } - - @Override - public void cleanup() { - TaskHistory taskHistory = createTaskHistory(); - executionBlockContext.addTaskHistory(taskRunnerId, getId(), taskHistory); - executionBlockContext.getTasks().remove(getId()); - - fetcherRunners.clear(); - fetcherRunners = null; - try { - if(executor != null) { - executor.close(); - executor = null; - } - } catch (IOException e) { - LOG.fatal(e.getMessage(), e); - } - - executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); - stopScriptExecutors(); - } - - public TaskHistory createTaskHistory() { - TaskHistory taskHistory = null; - try { - taskHistory = new TaskHistory(context.getTaskId(), context.getState(), context.getProgress(), - startTime, finishTime, reloadInputStats()); - - if (context.getOutputPath() != null) { - taskHistory.setOutputPath(context.getOutputPath().toString()); - } - - if (context.getWorkDir() != null) { - taskHistory.setWorkingPath(context.getWorkDir().toString()); - } - - if (context.getResultStats() != null) { - taskHistory.setOutputStats(context.getResultStats().getProto()); - } - - if (hasFetchPhase()) { - taskHistory.setTotalFetchCount(fetcherRunners.size()); - int i = 0; - FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); - for (Fetcher fetcher : fetcherRunners) { - // TODO store the fetcher histories - if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { - builder.setStartTime(fetcher.getStartTime()); - builder.setFinishTime(fetcher.getFinishTime()); - builder.setFileLength(fetcher.getFileLen()); - builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); - builder.setState(fetcher.getState()); - - taskHistory.addFetcherHistory(builder.build()); - } - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; - } - taskHistory.setFinishedFetchCount(i); - } - } catch (Exception e) { - LOG.warn(e.getMessage(), e); - } - - return taskHistory; - } - - public int hashCode() { - return context.hashCode(); - } - - public boolean equals(Object obj) { - if (obj instanceof LegacyTaskImpl) { - LegacyTaskImpl other = (LegacyTaskImpl) obj; - return this.context.equals(other.context); - } - return false; - } - - private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) - throws IOException { - Configuration c = new Configuration(systemConf); - c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); - FileSystem fs = FileSystem.get(c); - Path tablePath = new Path(file.getAbsolutePath()); - - List listTablets = new ArrayList(); - FileFragment tablet; - - FileStatus[] fileLists = fs.listStatus(tablePath); - for (FileStatus f : fileLists) { - if (f.getLen() == 0) { - continue; - } - tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); - listTablets.add(tablet); - } - - // Special treatment for locally pseudo fetched chunks - synchronized (localChunks) { - for (FileChunk chunk : localChunks) { - if (name.equals(chunk.getEbId())) { - tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); - listTablets.add(tablet); - LOG.info("One local chunk is added to listTablets"); - } - } - } - - FileFragment[] tablets = new FileFragment[listTablets.size()]; - listTablets.toArray(tablets); - - return tablets; - } - - private class FetchRunner implements Runnable { - private final TaskAttemptContext ctx; - private final Fetcher fetcher; - private int maxRetryNum; - - public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { - this.ctx = ctx; - this.fetcher = fetcher; - this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); - } - - @Override - public void run() { - int retryNum = 0; - int retryWaitTime = 1000; //sec - - try { // for releasing fetch latch - while(!context.isStopped() && retryNum < maxRetryNum) { - if (retryNum > 0) { - try { - Thread.sleep(retryWaitTime); - retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds - } catch (InterruptedException e) { - LOG.error(e); - } - LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); - } - try { - FileChunk fetched = fetcher.get(); - if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null - && fetched.getFile() != null) { - if (fetched.fromRemote() == false) { - localChunks.add(fetched); - LOG.info("Add a new FileChunk to local chunk list"); - } - break; - } - } catch (Throwable e) { - LOG.error("Fetch failed: " + fetcher.getURI(), e); - } - retryNum++; - } - } finally { - if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ - fetcherFinished(ctx); - } else { - if (retryNum == maxRetryNum) { - LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); - } - stopScriptExecutors(); - context.stop(); // retry task - ctx.getFetchLatch().countDown(); - } - } - } - } - - @VisibleForTesting - public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { - if (totalFetcher > 0) { - return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; - } else { - return 0.0f; - } - } - - private synchronized void fetcherFinished(TaskAttemptContext ctx) { - int fetcherSize = fetcherRunners.size(); - if(fetcherSize == 0) { - return; - } - - ctx.getFetchLatch().countDown(); - - int remainFetcher = (int) ctx.getFetchLatch().getCount(); - if (remainFetcher == 0) { - context.setFetcherProgress(FETCHER_PROGRESS); - } else { - context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); - } - } - - private List getFetchRunners(TaskAttemptContext ctx, - List fetches) throws IOException { - - if (fetches.size() > 0) { - Path inputDir = executionBlockContext.getLocalDirAllocator(). - getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); - - int i = 0; - File storeDir; - File defaultStoreFile; - FileChunk storeChunk = null; - List runnerList = Lists.newArrayList(); - - for (FetchImpl f : fetches) { - storeDir = new File(inputDir.toString(), f.getName()); - if (!storeDir.exists()) { - storeDir.mkdirs(); - } - - for (URI uri : f.getURIs()) { - defaultStoreFile = new File(storeDir, "in_" + i); - InetAddress address = InetAddress.getByName(uri.getHost()); - - WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); - if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { - boolean hasError = false; - try { - LOG.info("Try to get local file chunk at local host"); - storeChunk = getLocalStoredFileChunk(uri, systemConf); - } catch (Throwable t) { - hasError = true; - } - - // When a range request is out of range, storeChunk will be NULL. This case is normal state. - // So, we should skip and don't need to create storeChunk. - if (storeChunk == null && !hasError) { - continue; - } - - if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 - && hasError == false) { - storeChunk.setFromRemote(false); - } else { - storeChunk = new FileChunk(defaultStoreFile, 0, -1); - storeChunk.setFromRemote(true); - } - } else { - storeChunk = new FileChunk(defaultStoreFile, 0, -1); - storeChunk.setFromRemote(true); - } - - // If we decide that intermediate data should be really fetched from a remote host, storeChunk - // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it - storeChunk.setEbId(f.getName()); - Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); - LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); - runnerList.add(fetcher); - i++; - } - } - ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); - return runnerList; - } else { - return Lists.newArrayList(); - } - } - - private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { - // Parse the URI - LOG.info("getLocalStoredFileChunk starts"); - final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); - final List types = params.get("type"); - final List qids = params.get("qid"); - final List taskIdList = params.get("ta"); - final List stageIds = params.get("sid"); - final List partIds = params.get("p"); - final List offsetList = params.get("offset"); - final List lengthList = params.get("length"); - - if (types == null || stageIds == null || qids == null || partIds == null) { - LOG.error("Invalid URI - Required queryId, type, stage Id, and part id"); - return null; - } - - if (qids.size() != 1 && types.size() != 1 || stageIds.size() != 1) { - LOG.error("Invalid URI - Required qids, type, taskIds, stage Id, and part id"); - return null; - } - - String queryId = qids.get(0); - String shuffleType = types.get(0); - String sid = stageIds.get(0); - String partId = partIds.get(0); - - if (shuffleType.equals("r") && taskIdList == null) { - LOG.error("Invalid URI - For range shuffle, taskId is required"); - return null; - } - List taskIds = splitMaps(taskIdList); - - FileChunk chunk = null; - long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; - long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; - - LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId - + ", taskIds=" + taskIdList); - - // The working directory of Tajo worker for each query, including stage - String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; - - // If the stage requires a range shuffle - if (shuffleType.equals("r")) { - String ta = taskIds.get(0); - if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { - LOG.warn("Range shuffle - file not exist"); - return null; - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); - String startKey = params.get("start").get(0); - String endKey = params.get("end").get(0); - boolean last = params.get("final") != null; - - try { - chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); - } catch (Throwable t) { - LOG.error("getFileChunks() throws exception"); - return null; - } - - // If the stage requires a hash shuffle or a scattered hash shuffle - } else if (shuffleType.equals("h") || shuffleType.equals("s")) { - int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf); - String partPath = queryBaseDir + "hash-shuffle/" + partParentId + "/" + partId; - if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { - LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); - return null; - } - Path path = executionBlockContext.getLocalFS().makeQualified( - executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); - File file = new File(path.toUri()); - long startPos = (offset >= 0 && length >= 0) ? offset : 0; - long readLen = (offset >= 0 && length >= 0) ? length : file.length(); - - if (startPos >= file.length()) { - LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); - return null; - } - chunk = new FileChunk(file, startPos, readLen); - - } else { - LOG.error("Unknown shuffle type"); - return null; - } - - return chunk; - } - - private List splitMaps(List mapq) { - if (null == mapq) { - return null; - } - final List ret = new ArrayList(); - for (String s : mapq) { - Collections.addAll(ret, s.split(",")); - } - return ret; - } - - public static Path getTaskAttemptDir(TaskAttemptId quid) { - Path workDir = - StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), - String.valueOf(quid.getTaskId().getId()), - String.valueOf(quid.getId())); - return workDir; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index e763d13b4f..9156ae6206 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -30,6 +30,7 @@ import org.apache.tajo.storage.DiskUtil; import org.apache.tajo.unit.StorageUnit; import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.event.*; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; @@ -38,15 +39,15 @@ public class NodeResourceManager extends AbstractService implements EventHandler private static final Log LOG = LogFactory.getLog(NodeResourceManager.class); private final Dispatcher dispatcher; - private final EventHandler taskEventHandler; + private final TajoWorker.WorkerContext workerContext; private NodeResource totalResource; private NodeResource availableResource; private TajoConf tajoConf; - public NodeResourceManager(Dispatcher dispatcher, EventHandler taskEventHandler) { + public NodeResourceManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext) { super(NodeResourceManager.class.getName()); this.dispatcher = dispatcher; - this.taskEventHandler = taskEventHandler; + this.workerContext = workerContext; } @Override @@ -66,13 +67,23 @@ protected void serviceInit(Configuration conf) throws Exception { @Override public void handle(NodeResourceEvent event) { - if (event instanceof NodeResourceAllocateEvent) { + if (event instanceof QMResourceAllocateEvent) { + // allocate query master resource + QMResourceAllocateEvent allocateEvent = TUtil.checkTypeAndGet(event, QMResourceAllocateEvent.class); + NodeResource resource = new NodeResource(allocateEvent.getRequest().getResource()); + if (allocate(resource)) { + allocateEvent.getCallback().run(TajoWorker.TRUE_PROTO); + } else { + allocateEvent.getCallback().run(TajoWorker.FALSE_PROTO); + } + } else if (event instanceof NodeResourceAllocateEvent) { + // allocate task resource NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { - if(allocateEvent.getRequest().hasExecutionBlockRequest()){ + if (allocateEvent.getRequest().hasExecutionBlockRequest()) { //send ExecutionBlock start event to TaskManager startExecutionBlock(allocateEvent.getRequest().getExecutionBlockRequest()); } @@ -96,7 +107,7 @@ public void handle(NodeResourceEvent event) { } } - protected Dispatcher getDispatcher() { + public Dispatcher getDispatcher() { return dispatcher; } @@ -117,12 +128,12 @@ private boolean allocate(NodeResource resource) { return false; } - protected void startExecutionBlock(RunExecutionBlockRequestProto request) { - taskEventHandler.handle(new ExecutionBlockStartEvent(request)); + protected void startExecutionBlock(StartExecutionBlockRequestProto request) { + workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new ExecutionBlockStartEvent(request)); } protected void startTask(TaskRequestProto request, NodeResource resource) { - taskEventHandler.handle(new TaskStartEvent(request, resource)); + workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new TaskStartEvent(request, resource)); } private void release(NodeResource resource) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index d13cd50a32..a488108096 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -28,6 +28,7 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.resource.DefaultResourceCalculator; import org.apache.tajo.resource.NodeResource; import org.apache.tajo.rpc.*; import org.apache.tajo.service.ServiceTracker; @@ -58,16 +59,14 @@ public class NodeStatusUpdater extends AbstractService implements EventHandler heartBeatRequestQueue; private final TajoWorker.WorkerContext workerContext; - private final NodeResourceManager nodeResourceManager; private AsyncRpcClient rmClient; private ServiceTracker serviceTracker; private TajoResourceTrackerProtocolService.Interface resourceTracker; private int queueingLimit; - public NodeStatusUpdater(TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { + public NodeStatusUpdater(TajoWorker.WorkerContext workerContext) { super(NodeStatusUpdater.class.getSimpleName()); this.workerContext = workerContext; - this.nodeResourceManager = resourceManager; } @Override @@ -78,7 +77,7 @@ public void serviceInit(Configuration conf) throws Exception { this.tajoConf = (TajoConf) conf; this.heartBeatRequestQueue = Queues.newLinkedBlockingQueue(); this.serviceTracker = ServiceTrackerFactory.get(tajoConf); - this.nodeResourceManager.getDispatcher().register(NodeStatusEvent.EventType.class, this); + this.workerContext.getNodeResourceManager().getDispatcher().register(NodeStatusEvent.EventType.class, this); this.heartBeatInterval = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL); this.updaterThread = new StatusUpdaterThread(); super.serviceInit(conf); @@ -87,7 +86,11 @@ public void serviceInit(Configuration conf) throws Exception { @Override public void serviceStart() throws Exception { // if resource changed over than 50%, send reports - this.queueingLimit = nodeResourceManager.getTotalResource().getVirtualCores() / 2; + DefaultResourceCalculator calculator = new DefaultResourceCalculator(); + int maxContainer = calculator.computeAvailableContainers(workerContext.getNodeResourceManager().getTotalResource(), + NodeResource.createResource(512, 1, 1)); + this.queueingLimit = (int) Math.ceil(maxContainer / 2); + LOG.info("Queueing limit:" + queueingLimit); updaterThread.start(); super.serviceStart(); @@ -119,28 +122,26 @@ public int getQueueingLimit() { return queueingLimit; } - private NodeHeartbeatRequestProto createResourceReport(NodeResource resource) { + private NodeHeartbeatRequestProto.Builder createHeartBeatReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setAvailableResource(resource.getProto()); requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); - return requestProto.build(); + return requestProto; } - private NodeHeartbeatRequestProto createHeartBeatReport() { - NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); - return requestProto.build(); + private NodeHeartbeatRequestProto.Builder createResourceReport() { + NodeHeartbeatRequestProto.Builder requestProto = createHeartBeatReport(); + requestProto.setAvailableResource(workerContext.getNodeResourceManager().getAvailableResource().getProto()); + requestProto.setRunningTasks(workerContext.getTaskManager().getRunningTasks()); + return requestProto; } - private NodeHeartbeatRequestProto createNodeStatusReport() { - NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setTotalResource(nodeResourceManager.getTotalResource().getProto()); - requestProto.setAvailableResource(nodeResourceManager.getAvailableResource().getProto()); - requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); + private NodeHeartbeatRequestProto.Builder createNodeStatusReport() { + NodeHeartbeatRequestProto.Builder requestProto = createResourceReport(); + requestProto.setTotalResource(workerContext.getNodeResourceManager().getTotalResource().getProto()); requestProto.setConnectionInfo(workerContext.getConnectionInfo().getProto()); //TODO set node status to requestProto.setStatus() - return requestProto.build(); + return requestProto; } protected TajoResourceTrackerProtocolService.Interface newStub() @@ -226,21 +227,21 @@ public void run() { if (!events.isEmpty()) { // send current available resource; - lastResponse = sendHeartbeat(createResourceReport(nodeResourceManager.getAvailableResource())); + lastResponse = sendHeartbeat(createResourceReport().build()); } else { // send ping; - lastResponse = sendHeartbeat(createHeartBeatReport()); + lastResponse = sendHeartbeat(createHeartBeatReport().build()); } } else if (lastResponse.getCommand() == ResponseCommand.MEMBERSHIP) { // Membership changed - lastResponse = sendHeartbeat(createNodeStatusReport()); + lastResponse = sendHeartbeat(createNodeStatusReport().build()); } else if (lastResponse.getCommand() == ResponseCommand.ABORT_QUERY) { //TODO abort failure queries } } else { // Node registration on startup - lastResponse = sendHeartbeat(createNodeStatusReport()); + lastResponse = sendHeartbeat(createNodeStatusReport().build()); } } catch (NoSuchMethodException nsme) { LOG.fatal(nsme.getMessage(), nsme); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java index 05dd1a940b..62db0a44ab 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java @@ -32,9 +32,6 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.*; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.container.TajoContainer; @@ -98,12 +95,13 @@ public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext, int memoryMBPerTask) { //TODO consider disk slot - ClusterResourceSummary clusterResource = workerContext.getClusterResource(); - int clusterSlots = clusterResource == null ? 0 : clusterResource.getTotalMemoryMB() / memoryMBPerTask; - clusterSlots = Math.max(1, clusterSlots - 1); // reserve query master slot - LOG.info("CalculateNumberRequestContainer - Number of Tasks=" + numTasks + - ", Number of Cluster Slots=" + clusterSlots); - return Math.min(numTasks, clusterSlots); +// ClusterResourceSummary clusterResource = workerContext.getClusterResource(); +// int clusterSlots = clusterResource == null ? 0 : clusterResource.getTotalMemoryMB() / memoryMBPerTask; +// clusterSlots = Math.max(1, clusterSlots - 1); // reserve query master slot +// LOG.info("CalculateNumberRequestContainer - Number of Tasks=" + numTasks + +// ", Number of Cluster Slots=" + clusterSlots); +// return Math.min(numTasks, clusterSlots); + return 0; } @Override @@ -131,16 +129,16 @@ public synchronized void stop() { allocationExecutor.shutdownNow(); deallocator.shutdown(); - Map containers = queryTaskContext.getResourceAllocator() - .getContainers(); - List list = new ArrayList(containers.values()); - for(ContainerProxy eachProxy: list) { - try { - eachProxy.stopContainer(); - } catch (Throwable e) { - LOG.warn(e.getMessage(), e); - } - } +// Map containers = queryTaskContext.getResourceAllocator() +// .getContainers(); +// List list = new ArrayList(containers.values()); +// for(ContainerProxy eachProxy: list) { +// try { +// eachProxy.stopContainer(); +// } catch (Throwable e) { +// LOG.warn(e.getMessage(), e); +// } +// } workerInfoMap.clear(); super.stop(); @@ -195,17 +193,17 @@ public void run() { } private void stopExecutionBlock(ExecutionBlockId executionBlockId, NodeId worker) { - NettyClientBase tajoWorkerRpc = null; - try { - InetSocketAddress addr = new InetSocketAddress(worker.getHost(), worker.getPort()); - tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); - TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); - - tajoWorkerRpcClient.stopExecutionBlock(null, executionBlockId.getProto(), - NullCallback.get(PrimitiveProtos.BoolProto.class)); - } catch (Throwable e) { - LOG.error(e.getMessage(), e); - } +// NettyClientBase tajoWorkerRpc = null; +// try { +// InetSocketAddress addr = new InetSocketAddress(worker.getHost(), worker.getPort()); +// tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); +// TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); +// +// tajoWorkerRpcClient.stopExecutionBlock(null, executionBlockId.getProto(), +// NullCallback.get(PrimitiveProtos.BoolProto.class)); +// } catch (Throwable e) { +// LOG.error(e.getMessage(), e); +// } } protected static class LaunchRunner implements Runnable { @@ -251,8 +249,8 @@ private void shutdown() { @Override public void run() { - final AbstractResourceAllocator allocator = queryTaskContext.getResourceAllocator(); - while (!stopped.get() || !queue.isEmpty()) { + //final AbstractResourceAllocator allocator = queryTaskContext.getResourceAllocator(); + /*while (!stopped.get() || !queue.isEmpty()) { TajoContainerId containerId; try { containerId = queue.take(); @@ -272,7 +270,7 @@ public void run() { } catch (Exception e) { LOG.warn("Failed to stop container " + proxy.getContainerId() + "," + proxy.getBlockId(), e); } - } + }*/ LOG.info("Deallocator exiting"); } } @@ -293,123 +291,123 @@ class TajoWorkerAllocationThread extends Thread { @Override public void run() { LOG.info("Start TajoWorkerAllocationThread"); - CallFuture callBack = - new CallFuture(); +// CallFuture callBack = +// new CallFuture(); //TODO consider task's resource usage pattern int requiredMemoryMB = tajoConf.getIntVar(TajoConf.ConfVars.TASK_DEFAULT_MEMORY); float requiredDiskSlots = tajoConf.getFloatVar(TajoConf.ConfVars.TASK_DEFAULT_DISK); - WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder() - .setMinMemoryMBPerContainer(requiredMemoryMB) - .setMaxMemoryMBPerContainer(requiredMemoryMB) - .setNumContainers(event.getRequiredNum()) - .setResourceRequestPriority(!event.isLeafQuery() ? - ResourceRequestPriority.MEMORY : ResourceRequestPriority.DISK) - .setMinDiskSlotPerContainer(requiredDiskSlots) - .setMaxDiskSlotPerContainer(requiredDiskSlots) - .setQueryId(event.getExecutionBlockId().getQueryId().getProto()) - .build(); - - - NettyClientBase tmClient = null; - try { - ServiceTracker serviceTracker = queryTaskContext.getQueryMasterContext().getWorkerContext().getServiceTracker(); - tmClient = RpcClientManager.getInstance(). - getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); - QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); - masterClientService.allocateWorkerResources(callBack.getController(), request, callBack); - } catch (Throwable e) { - LOG.error(e.getMessage(), e); - } - - WorkerResourceAllocationResponse response = null; - while(!stopped.get()) { - try { - response = callBack.get(3, TimeUnit.SECONDS); - break; - } catch (InterruptedException e) { - if(stopped.get()) { - return; - } - } catch (TimeoutException e) { - LOG.info("No available worker resource for " + event.getExecutionBlockId()); - continue; - } catch (ExecutionException e) { - LOG.error(e.getMessage(), e); - break; - } - } - - int numAllocatedContainers = 0; - - if(response != null) { - List allocatedResources = response.getWorkerAllocatedResourceList(); - ExecutionBlockId executionBlockId = event.getExecutionBlockId(); - - List containers = new ArrayList(); - for(WorkerAllocatedResource eachAllocatedResource: allocatedResources) { - TajoWorkerContainer container = new TajoWorkerContainer(); - NodeId nodeId = NodeId.newInstance(eachAllocatedResource.getConnectionInfo().getHost(), - eachAllocatedResource.getConnectionInfo().getPeerRpcPort()); - - TajoWorkerContainerId containerId = new TajoWorkerContainerId(); - - containerId.setApplicationAttemptId( - ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId(), - eachAllocatedResource.getContainerId().getAppAttemptId().getAttemptId())); - containerId.setId(eachAllocatedResource.getContainerId().getId()); - - container.setId(containerId); - container.setNodeId(nodeId); - - - WorkerResource workerResource = new WorkerResource(); - workerResource.setMemoryMB(eachAllocatedResource.getAllocatedMemoryMB()); - workerResource.setDiskSlots(eachAllocatedResource.getAllocatedDiskSlots()); - - Worker worker = new Worker(null, workerResource, - new WorkerConnectionInfo(eachAllocatedResource.getConnectionInfo())); - container.setWorkerResource(worker); - addWorkerConnectionInfo(worker.getConnectionInfo()); - containers.add(container); - } - - StageState state = queryTaskContext.getStage(executionBlockId).getSynchronizedState(); - if (!Stage.isRunningState(state)) { - List containerIds = new ArrayList(); - for(TajoContainer eachContainer: containers) { - containerIds.add(eachContainer.getId()); - } - try { - TajoContainerProxy.releaseWorkerResource(queryTaskContext, executionBlockId, containerIds); - } catch (Throwable e) { - deallocator.submit(containerIds); - LOG.error(e.getMessage(), e); - } - return; - } - - if (allocatedResources.size() > 0) { - if(LOG.isDebugEnabled()) { - LOG.debug("StageContainerAllocationEvent fire:" + executionBlockId); - } - queryTaskContext.getEventHandler().handle(new StageContainerAllocationEvent(executionBlockId, containers)); - } - numAllocatedContainers += allocatedResources.size(); - - } - if(event.getRequiredNum() > numAllocatedContainers) { - ContainerAllocationEvent shortRequestEvent = new ContainerAllocationEvent( - event.getType(), event.getExecutionBlockId(), event.getPriority(), - event.getResource(), - event.getRequiredNum() - numAllocatedContainers, - event.isLeafQuery(), event.getProgress() - ); - queryTaskContext.getEventHandler().handle(shortRequestEvent); - - } - LOG.info("Stop TajoWorkerAllocationThread"); +// WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder() +// .setMinMemoryMBPerContainer(requiredMemoryMB) +// .setMaxMemoryMBPerContainer(requiredMemoryMB) +// .setNumContainers(event.getRequiredNum()) +// .setResourceRequestPriority(!event.isLeafQuery() ? +// ResourceRequestPriority.MEMORY : ResourceRequestPriority.DISK) +// .setMinDiskSlotPerContainer(requiredDiskSlots) +// .setMaxDiskSlotPerContainer(requiredDiskSlots) +// .setQueryId(event.getExecutionBlockId().getQueryId().getProto()) +// .build(); +// +// +// NettyClientBase tmClient = null; +// try { +// ServiceTracker serviceTracker = queryTaskContext.getQueryMasterContext().getWorkerContext().getServiceTracker(); +// tmClient = RpcClientManager.getInstance(). +// getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); +// QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); +// masterClientService.allocateWorkerResources(callBack.getController(), request, callBack); +// } catch (Throwable e) { +// LOG.error(e.getMessage(), e); +// } +// +// WorkerResourceAllocationResponse response = null; +// while(!stopped.get()) { +// try { +// response = callBack.get(3, TimeUnit.SECONDS); +// break; +// } catch (InterruptedException e) { +// if(stopped.get()) { +// return; +// } +// } catch (TimeoutException e) { +// LOG.info("No available worker resource for " + event.getExecutionBlockId()); +// continue; +// } catch (ExecutionException e) { +// LOG.error(e.getMessage(), e); +// break; +// } +// } +// +// int numAllocatedContainers = 0; +// +// if(response != null) { +// List allocatedResources = response.getWorkerAllocatedResourceList(); +// ExecutionBlockId executionBlockId = event.getExecutionBlockId(); +// +// List containers = new ArrayList(); +// for(WorkerAllocatedResource eachAllocatedResource: allocatedResources) { +// TajoWorkerContainer container = new TajoWorkerContainer(); +// NodeId nodeId = NodeId.newInstance(eachAllocatedResource.getConnectionInfo().getHost(), +// eachAllocatedResource.getConnectionInfo().getPeerRpcPort()); +// +// TajoWorkerContainerId containerId = new TajoWorkerContainerId(); +// +// containerId.setApplicationAttemptId( +// ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId(), +// eachAllocatedResource.getContainerId().getAppAttemptId().getAttemptId())); +// containerId.setId(eachAllocatedResource.getContainerId().getId()); +// +// container.setId(containerId); +// container.setNodeId(nodeId); +// +// +// WorkerResource workerResource = new WorkerResource(); +// workerResource.setMemoryMB(eachAllocatedResource.getAllocatedMemoryMB()); +// workerResource.setDiskSlots(eachAllocatedResource.getAllocatedDiskSlots()); +// +// Worker worker = new Worker(null, workerResource, +// new WorkerConnectionInfo(eachAllocatedResource.getConnectionInfo())); +// container.setWorkerResource(worker); +// addWorkerConnectionInfo(worker.getConnectionInfo()); +// containers.add(container); +// } +// +// StageState state = queryTaskContext.getStage(executionBlockId).getSynchronizedState(); +// if (!Stage.isRunningState(state)) { +// List containerIds = new ArrayList(); +// for(TajoContainer eachContainer: containers) { +// containerIds.add(eachContainer.getId()); +// } +// try { +// TajoContainerProxy.releaseWorkerResource(queryTaskContext, executionBlockId, containerIds); +// } catch (Throwable e) { +// deallocator.submit(containerIds); +// LOG.error(e.getMessage(), e); +// } +// return; +// } +// +// if (allocatedResources.size() > 0) { +// if(LOG.isDebugEnabled()) { +// LOG.debug("StageContainerAllocationEvent fire:" + executionBlockId); +// } +// queryTaskContext.getEventHandler().handle(new StageContainerAllocationEvent(executionBlockId, containers)); +// } +// numAllocatedContainers += allocatedResources.size(); +// +// } +// if(event.getRequiredNum() > numAllocatedContainers) { +// ContainerAllocationEvent shortRequestEvent = new ContainerAllocationEvent( +// event.getType(), event.getExecutionBlockId(), event.getPriority(), +// event.getResource(), +// event.getRequiredNum() - numAllocatedContainers, +// event.isLeafQuery(), event.getProgress() +// ); +// queryTaskContext.getEventHandler().handle(shortRequestEvent); +// +// } +// LOG.info("Stop TajoWorkerAllocationThread"); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 010a11d40c..7b87741a45 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -42,9 +42,7 @@ import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.service.TajoMasterInfo; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.ClusterResourceSummary; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.querymaster.QueryMaster; import org.apache.tajo.querymaster.QueryMasterManagerService; @@ -97,19 +95,19 @@ public class TajoWorker extends CompositeService { private WorkerContext workerContext; - private TaskRunnerManager taskRunnerManager; + private TaskManager taskManager; + + private TaskExecutor taskExecutor; private TajoPullServerService pullService; private ServiceTracker serviceTracker; - private WorkerHeartbeatService workerHeartbeatThread; - - private AtomicBoolean stopped = new AtomicBoolean(false); + private NodeResourceManager nodeResourceManager; - private AtomicInteger numClusterNodes = new AtomicInteger(); + private NodeStatusUpdater nodeStatusUpdater; - private ClusterResourceSummary clusterResource; + private AtomicBoolean stopped = new AtomicBoolean(false); private WorkerConnectionInfo connectionInfo; @@ -165,23 +163,11 @@ public void serviceInit(Configuration conf) throws Exception { this.workerContext = new TajoWorkerContext(); this.lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); - String resourceManagerClassName = systemConf.getVar(ConfVars.RESOURCE_MANAGER_CLASS); - - boolean randomPort = true; - if(resourceManagerClassName.indexOf(TajoResourceManager.class.getName()) >= 0) { - randomPort = false; - } int clientPort = systemConf.getSocketAddrVar(ConfVars.WORKER_CLIENT_RPC_ADDRESS).getPort(); int peerRpcPort = systemConf.getSocketAddrVar(ConfVars.WORKER_PEER_RPC_ADDRESS).getPort(); int qmManagerPort = systemConf.getSocketAddrVar(ConfVars.WORKER_QM_RPC_ADDRESS).getPort(); - if(randomPort) { - clientPort = 0; - peerRpcPort = 0; - qmManagerPort = 0; - systemConf.setIntVar(ConfVars.PULLSERVER_PORT, 0); - } this.dispatcher = new AsyncDispatcher(); addIfService(dispatcher); @@ -196,12 +182,19 @@ public void serviceInit(Configuration conf) throws Exception { queryMasterManagerService = new QueryMasterManagerService(workerContext, qmManagerPort); addIfService(queryMasterManagerService); - // taskrunner worker - taskRunnerManager = new TaskRunnerManager(workerContext, dispatcher); - addService(taskRunnerManager); + this.taskManager = new TaskManager(dispatcher, workerContext); + addService(taskManager); + + this.taskExecutor = new TaskExecutor(workerContext); + addService(taskExecutor); - workerHeartbeatThread = new WorkerHeartbeatService(workerContext); - addIfService(workerHeartbeatThread); + AsyncDispatcher rmDispatcher = new AsyncDispatcher(); + addService(rmDispatcher); + this.nodeResourceManager = new NodeResourceManager(rmDispatcher, workerContext); + addService(nodeResourceManager); + + this.nodeStatusUpdater = new NodeStatusUpdater(workerContext); + addService(nodeStatusUpdater); int httpPort = 0; if(!TajoPullServerService.isStandalone()) { @@ -268,8 +261,8 @@ public Integer getValue() { workerSystemMetrics.register("task", "runningTasks", new Gauge() { @Override public Integer getValue() { - if(taskRunnerManager != null) { - return taskRunnerManager.getNumTasks(); + if(taskExecutor != null) { + return taskExecutor.getRunningTasks(); } else { return 0; } @@ -394,7 +387,11 @@ public interface WorkerContext { QueryMasterManagerService getQueryMasterManagerService(); - TaskRunnerManager getTaskRunnerManager(); + TaskManager getTaskManager(); + + TaskExecutor getTaskExecuor(); + + NodeResourceManager getNodeResourceManager(); CatalogService getCatalog(); @@ -404,8 +401,6 @@ public interface WorkerContext { LocalDirAllocator getLocalDirAllocator(); - ClusterResourceSummary getClusterResource(); - TajoSystemMetrics getWorkerSystemMetrics(); HashShuffleAppenderManager getHashShuffleAppenderManager(); @@ -417,10 +412,6 @@ public interface WorkerContext { void cleanup(String strPath); void cleanupTemporalDirectories(); - - void setClusterResource(ClusterResourceSummary clusterResource); - - void setNumClusterNodes(int numClusterNodes); } class TajoWorkerContext implements WorkerContext { @@ -443,8 +434,19 @@ public QueryMasterManagerService getQueryMasterManagerService() { return queryMasterManagerService; } - public TaskRunnerManager getTaskRunnerManager() { - return taskRunnerManager; + @Override + public TaskManager getTaskManager(){ + return taskManager; + } + + @Override + public TaskExecutor getTaskExecuor() { + return taskExecutor; + } + + @Override + public NodeResourceManager getNodeResourceManager() { + return nodeResourceManager; } public CatalogService getCatalog() { @@ -507,22 +509,6 @@ public void cleanupTemporalDirectories() { } } - public void setNumClusterNodes(int numClusterNodes) { - TajoWorker.this.numClusterNodes.set(numClusterNodes); - } - - public void setClusterResource(ClusterResourceSummary clusterResource) { - synchronized (numClusterNodes) { - TajoWorker.this.clusterResource = clusterResource; - } - } - - public ClusterResourceSummary getClusterResource() { - synchronized (numClusterNodes) { - return TajoWorker.this.clusterResource; - } - } - public TajoSystemMetrics getWorkerSystemMetrics() { return workerSystemMetrics; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index de8afe8377..85b769634f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -30,14 +30,12 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TajoIdProtos; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.util.NetUtils; -import org.apache.tajo.worker.event.TaskRunnerStartEvent; -import org.apache.tajo.worker.event.TaskRunnerStopEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; import java.net.InetSocketAddress; @@ -110,29 +108,39 @@ public void ping(RpcController controller, } @Override - public void startExecutionBlock(RpcController controller, - TajoWorkerProtocol.RunExecutionBlockRequestProto request, - RpcCallback done) { - workerContext.getWorkerSystemMetrics().counter("query", "executedExecutionBlocksNum").inc(); - - try { - workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStartEvent(request)); - done.run(TajoWorker.TRUE_PROTO); - } catch (Throwable t) { - LOG.error(t.getMessage(), t); - controller.setFailed(t.getMessage()); - done.run(TajoWorker.FALSE_PROTO); - } + public void allocateTasks(RpcController controller, + TajoWorkerProtocol.BatchAllocationRequestProto request, + RpcCallback done) { + //LOG.info("Try to allocate Tasks : " + request.toString()); + workerContext.getNodeResourceManager().getDispatcher(). + getEventHandler().handle(new NodeResourceAllocateEvent(request, done)); } +// @Override +// public void startExecutionBlock(RpcController controller, +// TajoWorkerProtocol.StartExecutionBlockRequestProto request, +// RpcCallback done) { +// workerContext.getWorkerSystemMetrics().counter("query", "executedExecutionBlocksNum").inc(); +// +// try { +// workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new TaskRunnerStartEvent(request)); +// done.run(TajoWorker.TRUE_PROTO); +// } catch (Throwable t) { +// LOG.error(t.getMessage(), t); +// controller.setFailed(t.getMessage()); +// done.run(TajoWorker.FALSE_PROTO); +// } +// } + @Override public void stopExecutionBlock(RpcController controller, - TajoIdProtos.ExecutionBlockIdProto requestProto, + TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto, RpcCallback done) { try { - workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStopEvent( - new ExecutionBlockId(requestProto) - )); + + workerContext.getTaskManager().getDispatcher().getEventHandler().handle( + new ExecutionBlockStopEvent(requestProto.getExecutionBlockId(), requestProto.getChild())); + done.run(TajoWorker.TRUE_PROTO); } catch (Exception e) { LOG.error(e.getMessage(), e); @@ -144,7 +152,8 @@ public void stopExecutionBlock(RpcController controller, @Override public void killTaskAttempt(RpcController controller, TajoIdProtos.TaskAttemptIdProto request, RpcCallback done) { - Task task = workerContext.getTaskRunnerManager().getTaskByTaskAttemptId(new TaskAttemptId(request)); + //TODO change to async ? + Task task = workerContext.getTaskManager().getTaskByTaskAttemptId(new TaskAttemptId(request)); if(task != null) task.kill(); done.run(TajoWorker.TRUE_PROTO); @@ -157,6 +166,7 @@ public void cleanup(RpcController controller, TajoIdProtos.QueryIdProto request, done.run(TajoWorker.TRUE_PROTO); } + @Deprecated @Override public void cleanupExecutionBlocks(RpcController controller, TajoWorkerProtocol.ExecutionBlockListProto ebIds, diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 299952e7ae..dd21e222db 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -49,8 +49,7 @@ public class TaskExecutor extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskExecutor.class); - private final TaskManager taskManager; - private final EventHandler rmEventHandler; + private final TajoWorker.WorkerContext workerContext; private final Map allocatedResourceMap; private final BlockingQueue taskQueue; private final AtomicInteger runningTasks; @@ -59,10 +58,9 @@ public class TaskExecutor extends AbstractService implements EventHandler(); @@ -75,7 +73,7 @@ protected void serviceInit(Configuration conf) throws Exception { } this.tajoConf = (TajoConf) conf; - this.taskManager.getDispatcher().register(TaskExecutorEvent.EventType.class, this); + this.workerContext.getTaskManager().getDispatcher().register(TaskExecutorEvent.EventType.class, this); super.serviceInit(conf); } @@ -134,7 +132,8 @@ protected Task getNextTask() { @SuppressWarnings("unchecked") protected void stopTask(TaskAttemptId taskId) { runningTasks.decrementAndGet(); - rmEventHandler.handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId))); + workerContext.getNodeResourceManager().getDispatcher().getEventHandler() + .handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId))); } protected ExecutorService getFetcherExecutor() { @@ -164,7 +163,7 @@ public void handle(TaskExecutorEvent event) { TaskStartEvent startEvent = (TaskStartEvent) event; allocatedResourceMap.put(startEvent.getTaskId(), startEvent.getAllocatedResource()); - ExecutionBlockContext context = taskManager.getExecutionBlockContext( + ExecutionBlockContext context = workerContext.getTaskManager().getExecutionBlockContext( startEvent.getTaskId().getTaskId().getExecutionBlockId()); try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index 7990a7213e..b767cd6363 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -45,17 +45,15 @@ public class TaskManager extends AbstractService implements EventHandler executionBlockContextMap; private final Dispatcher dispatcher; - private final EventHandler rmEventHandler; private TajoConf tajoConf; - public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext) { super(TaskManager.class.getName()); this.dispatcher = dispatcher; this.workerContext = workerContext; this.executionBlockContextMap = Maps.newHashMap(); - this.rmEventHandler = rmEventHandler; } @Override @@ -87,9 +85,13 @@ protected TajoWorker.WorkerContext getWorkerContext() { return workerContext; } - protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + public int getRunningTasks(){ + return workerContext.getTaskExecuor().getRunningTasks(); + } + + protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.StartExecutionBlockRequestProto request) { try { - ExecutionBlockContext context = new ExecutionBlockContext(getWorkerContext(), null, request); + ExecutionBlockContext context = new ExecutionBlockContext(getWorkerContext(), request); context.init(); return context; @@ -102,7 +104,7 @@ protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecu protected void stopExecutionBlock(ExecutionBlockContext context, TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { - if(context != null){ + if (context != null) { try { context.getSharedResource().releaseBroadcastCache(context.getExecutionBlockId()); context.sendShuffleReport(); @@ -135,12 +137,16 @@ public void handle(TaskManagerEvent event) { if(!executionBlockContextMap.containsKey(event.getExecutionBlockId())) { ExecutionBlockContext context = createExecutionBlock(((ExecutionBlockStartEvent) event).getRequestProto()); executionBlockContextMap.put(context.getExecutionBlockId(), context); + LOG.info("Running ExecutionBlocks: " + executionBlockContextMap.size() + + ", running tasks:" + getRunningTasks() + ", resource: " + + workerContext.getNodeResourceManager().getAvailableResource()); } else { LOG.warn("Already initialized ExecutionBlock: " + event.getExecutionBlockId()); } } else if (event instanceof ExecutionBlockStopEvent) { //receive event from QueryMaster - rmEventHandler.handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + workerContext.getNodeResourceManager().getDispatcher().getEventHandler() + .handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); stopExecutionBlock(executionBlockContextMap.remove(event.getExecutionBlockId()), ((ExecutionBlockStopEvent) event).getCleanupList()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java deleted file mode 100644 index 207b47e9f8..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java +++ /dev/null @@ -1,306 +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.worker; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.service.AbstractService; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.engine.query.TaskRequestImpl; -import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService; -import org.apache.tajo.master.container.TajoContainerId; -import org.apache.tajo.master.container.TajoContainerIdPBImpl; -import org.apache.tajo.master.container.TajoConverterUtils; -import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NullCallback; - -import java.util.concurrent.*; - -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; - -/** - * The driver class for Tajo Task processing. - */ -@Deprecated -public class TaskRunner extends AbstractService { - /** class logger */ - private static final Log LOG = LogFactory.getLog(TaskRunner.class); - - private TajoConf systemConf; - - private volatile boolean stopped = false; - private Path baseDirPath; - - private TajoContainerId containerId; - - // for Fetcher - private ExecutorService fetchLauncher; - - // A thread to receive each assigned query unit and execute the query unit - private Thread taskLauncher; - - // Contains the object references related for TaskRunner - private ExecutionBlockContext executionBlockContext; - - private long finishTime; - - private TaskRunnerHistory history; - - public TaskRunner(ExecutionBlockContext executionBlockContext, String containerId) { - super(TaskRunner.class.getName()); - - ThreadFactoryBuilder builder = new ThreadFactoryBuilder(); - ThreadFactory fetcherFactory = builder.setNameFormat("Fetcher executor #%d").build(); - this.systemConf = executionBlockContext.getConf(); - this.fetchLauncher = Executors.newFixedThreadPool( - systemConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM), fetcherFactory); - try { - this.containerId = TajoConverterUtils.toTajoContainerId(containerId); - this.executionBlockContext = executionBlockContext; - this.history = executionBlockContext.createTaskRunnerHistory(this); - this.history.setState(getServiceState()); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - } - - // TODO this is expensive. we should change to unique id - public String getId() { - return getId(getContext().getExecutionBlockId(), containerId); - } - - public TajoContainerId getContainerId(){ - return containerId; - } - - public static String getId(ExecutionBlockId executionBlockId, TajoContainerId containerId) { - return executionBlockId + "," + containerId; - } - - public TaskRunnerHistory getHistory(){ - return history; - } - - public Path getTaskBaseDir(){ - return baseDirPath; - } - - public ExecutorService getFetchLauncher() { - return fetchLauncher; - } - - @Override - public void init(Configuration conf) { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("conf should be a TajoConf Type."); - } - this.systemConf = (TajoConf)conf; - - try { - // the base dir for an output dir - baseDirPath = getContext().createBaseDir(); - LOG.info("TaskRunner basedir is created (" + baseDirPath +")"); - } catch (Throwable t) { - t.printStackTrace(); - LOG.error(t, t); - } - super.init(conf); - this.history.setState(getServiceState()); - } - - @Override - public void start() { - super.start(); - history.setStartTime(getStartTime()); - this.history.setState(getServiceState()); - run(); - } - - @Override - public void stop() { - if(isStopped()) { - return; - } - this.finishTime = System.currentTimeMillis(); - this.history.setFinishTime(finishTime); - // If this flag become true, taskLauncher will be terminated. - - LOG.info("Stop TaskRunner: " + getId()); - synchronized (this) { - this.stopped = true; - - fetchLauncher.shutdown(); - fetchLauncher = null; - - notifyAll(); - } - - super.stop(); - this.history.setState(getServiceState()); - } - - public long getFinishTime() { - return finishTime; - } - - public ExecutionBlockContext getContext() { - return executionBlockContext; - } - - static void fatalError(QueryMasterProtocolService.Interface qmClientService, - TaskAttemptId taskAttemptId, String message) { - if (message == null) { - message = "No error message"; - } - TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder() - .setId(taskAttemptId.getProto()) - .setErrorMessage(message); - - qmClientService.fatalError(null, builder.build(), NullCallback.get()); - } - - public void run() { - LOG.info("TaskRunner startup"); - try { - - taskLauncher = new Thread(new Runnable() { - - @Override - public void run() { - int receivedNum = 0; - CallFuture callFuture = null; - TaskRequestProto taskRequest = null; - - while(!stopped && !executionBlockContext.isStopped()) { - QueryMasterProtocolService.Interface qmClientService = executionBlockContext.getStub(); - - try { - if (callFuture == null) { - callFuture = new CallFuture(); - LOG.info("Request GetTask: " + getId()); - GetTaskRequestProto request = GetTaskRequestProto.newBuilder() - .setExecutionBlockId(getExecutionBlockId().getProto()) - .setContainerId(((TajoContainerIdPBImpl) containerId).getProto()) - .setWorkerId(getContext().getWorkerContext().getConnectionInfo().getId()) - .build(); - - qmClientService.getTask(callFuture.getController(), 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. - if (LOG.isDebugEnabled()) { - LOG.info("Retry assigning task:" + getId()); - } - continue; - } catch (ExecutionException ee) { - if(!getContext().isStopped()){ - LOG.error(ee.getMessage(), ee); - } else { - /* EB is stopped */ - break; - } - } - - if (taskRequest != null) { - // QueryMaster can send the terminal signal to TaskRunner. - // If TaskRunner receives the terminal signal, TaskRunner will be terminated - // immediately. - if (taskRequest.getShouldDie()) { - LOG.info("Received ShouldDie flag:" + getId()); - break; - } else { - getContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); - LOG.info("Accumulated Received Task: " + (++receivedNum)); - - TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); - if (getContext().getTasks().containsKey(taskAttemptId)) { - LOG.error("Duplicate Task Attempt: " + taskAttemptId); - fatalError(qmClientService, taskAttemptId, "Duplicate Task Attempt: " + taskAttemptId); - continue; - } - - LOG.info("Initializing: " + taskAttemptId); - Task task = null; - try { - task = new LegacyTaskImpl(getId(), getTaskBaseDir(), taskAttemptId, executionBlockContext, - new TaskRequestImpl(taskRequest)); - getContext().getTasks().put(taskAttemptId, task); - - task.init(); - if (task.hasFetchPhase()) { - task.fetch(); // The fetch is performed in an asynchronous way. - } - // task.run() is a blocking call. - task.run(); - } catch (Throwable t) { - LOG.error(t.getMessage(), t); - fatalError(qmClientService, taskAttemptId, t.getMessage()); - } finally { - if(task != null) { - task.cleanup(); - } - - callFuture = null; - taskRequest = null; - } - } - } - } catch (Throwable t) { - LOG.fatal(t.getMessage(), t); - } - } - stop(); - //notify to TaskRunnerManager - getContext().stopTaskRunner(getId()); - } - }); - taskLauncher.start(); - } catch (Throwable t) { - LOG.fatal("Unhandled exception. Starting shutdown.", t); - } - } - - /** - * @return true if a stop has been requested. - */ - public boolean isStopped() { - return this.stopped; - } - - public ExecutionBlockId getExecutionBlockId() { - return getContext().getExecutionBlockId(); - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java deleted file mode 100644 index d18a26295e..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ /dev/null @@ -1,238 +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.worker; - -import com.google.common.collect.Maps; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.CompositeService; -import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.worker.event.TaskRunnerEvent; -import org.apache.tajo.worker.event.TaskRunnerStartEvent; -import org.apache.tajo.worker.event.TaskRunnerStopEvent; - -import java.util.*; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicBoolean; - -@Deprecated -public class TaskRunnerManager extends CompositeService implements EventHandler { - private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class); - - private final ConcurrentMap executionBlockContextMap = Maps.newConcurrentMap(); - private final ConcurrentMap taskRunnerMap = Maps.newConcurrentMap(); - private final ConcurrentMap taskRunnerHistoryMap = Maps.newConcurrentMap(); - private TajoWorker.WorkerContext workerContext; - private TajoConf tajoConf; - private AtomicBoolean stop = new AtomicBoolean(false); - private FinishedTaskCleanThread finishedTaskCleanThread; - private Dispatcher dispatcher; - - public TaskRunnerManager(TajoWorker.WorkerContext workerContext, Dispatcher dispatcher) { - super(TaskRunnerManager.class.getName()); - - this.workerContext = workerContext; - this.dispatcher = dispatcher; - } - - public TajoWorker.WorkerContext getWorkerContext() { - return workerContext; - } - - @Override - public void init(Configuration conf) { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - tajoConf = (TajoConf)conf; - dispatcher.register(TaskRunnerEvent.EventType.class, this); - super.init(tajoConf); - } - - @Override - public void start() { - finishedTaskCleanThread = new FinishedTaskCleanThread(); - finishedTaskCleanThread.start(); - super.start(); - } - - @Override - public void stop() { - if(stop.getAndSet(true)) { - return; - } - - synchronized(taskRunnerMap) { - for(TaskRunner eachTaskRunner: taskRunnerMap.values()) { - if(!eachTaskRunner.isStopped()) { - eachTaskRunner.stop(); - } - } - } - for(ExecutionBlockContext context: executionBlockContextMap.values()) { - context.stop(); - } - - if(finishedTaskCleanThread != null) { - finishedTaskCleanThread.interrupt(); - } - - super.stop(); - } - - public void stopTaskRunner(String id) { - LOG.info("Stop Task:" + id); - TaskRunner taskRunner = taskRunnerMap.remove(id); - taskRunner.stop(); - } - - public Collection getTaskRunners() { - return Collections.unmodifiableCollection(taskRunnerMap.values()); - } - - public Collection getExecutionBlockHistories() { - return Collections.unmodifiableCollection(taskRunnerHistoryMap.values()); - } - - public TaskRunnerHistory getExcutionBlockHistoryByTaskRunnerId(String taskRunnerId) { - return taskRunnerHistoryMap.get(taskRunnerId); - } - - public TaskRunner getTaskRunner(String taskRunnerId) { - return taskRunnerMap.get(taskRunnerId); - } - - public Task getTaskByTaskAttemptId(TaskAttemptId taskAttemptId) { - ExecutionBlockContext context = executionBlockContextMap.get(taskAttemptId.getTaskId().getExecutionBlockId()); - if (context != null) { - return context.getTask(taskAttemptId); - } - return null; - } - - public TaskHistory getTaskHistoryByTaskAttemptId(TaskAttemptId quAttemptId) { - synchronized (taskRunnerHistoryMap) { - for (TaskRunnerHistory history : taskRunnerHistoryMap.values()) { - TaskHistory taskHistory = history.getTaskHistory(quAttemptId); - if (taskHistory != null) return taskHistory; - } - } - - return null; - } - - public int getNumTasks() { - return taskRunnerMap.size(); - } - - @Override - public void handle(TaskRunnerEvent event) { - LOG.info("======================== Processing " + event.getExecutionBlockId() + " of type " + event.getType()); - if (event instanceof TaskRunnerStartEvent) { - TaskRunnerStartEvent startEvent = (TaskRunnerStartEvent) event; - ExecutionBlockContext context = executionBlockContextMap.get(event.getExecutionBlockId()); - - if(context == null){ - try { - context = new ExecutionBlockContext(getWorkerContext(), this, startEvent.getRequest()); - context.init(); - } catch (Throwable e) { - LOG.fatal(e.getMessage(), e); - throw new RuntimeException(e); - } - executionBlockContextMap.put(event.getExecutionBlockId(), context); - } - - TaskRunner taskRunner = new TaskRunner(context, startEvent.getRequest().getContainerId()); - LOG.info("Start TaskRunner:" + taskRunner.getId()); - taskRunnerMap.put(taskRunner.getId(), taskRunner); - taskRunnerHistoryMap.put(taskRunner.getId(), taskRunner.getHistory()); - - taskRunner.init(context.getConf()); - taskRunner.start(); - - } else if (event instanceof TaskRunnerStopEvent) { - ExecutionBlockContext executionBlockContext = executionBlockContextMap.remove(event.getExecutionBlockId()); - if(executionBlockContext != null){ - try { - executionBlockContext.getSharedResource().releaseBroadcastCache(event.getExecutionBlockId()); - executionBlockContext.sendShuffleReport(); - workerContext.getTaskHistoryWriter().flushTaskHistories(); - } catch (Exception e) { - LOG.fatal(e.getMessage(), e); - throw new RuntimeException(e); - } finally { - executionBlockContext.stop(); - } - } - LOG.info("Stopped execution block:" + event.getExecutionBlockId()); - } - } - - public EventHandler getEventHandler(){ - return dispatcher.getEventHandler(); - } - - public TajoConf getTajoConf() { - return tajoConf; - } - - class FinishedTaskCleanThread extends Thread { - //TODO if history size is large, the historyMap should remove immediately - public void run() { - int expireIntervalTime = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HISTORY_EXPIRE_PERIOD); - LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime); - while(!stop.get()) { - try { - Thread.sleep(60 * 1000); - } catch (InterruptedException e) { - break; - } - try { - long expireTime = System.currentTimeMillis() - expireIntervalTime * 60 * 1000l; - cleanExpiredFinishedQueryMasterTask(expireTime); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - } - } - - private void cleanExpiredFinishedQueryMasterTask(long expireTime) { - synchronized(taskRunnerHistoryMap) { - List expiredIds = new ArrayList(); - for(Map.Entry entry: taskRunnerHistoryMap.entrySet()) { - /* If a task runner are abnormal termination, the finished time will be zero. */ - long finishedTime = Math.max(entry.getValue().getStartTime(), entry.getValue().getFinishTime()); - if(finishedTime < expireTime) { - expiredIds.add(entry.getKey()); - } - } - - for(String eachId: expiredIds) { - taskRunnerHistoryMap.remove(eachId); - } - } - } - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java b/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java deleted file mode 100644 index 050e2b59c7..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java +++ /dev/null @@ -1,262 +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.worker; - -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.service.AbstractService; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.ServerStatusProto; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeatResponse; -import org.apache.tajo.ipc.TajoResourceTrackerProtocol; -import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NettyClientBase; -import org.apache.tajo.rpc.RpcClientManager; -import org.apache.tajo.service.ServiceTracker; -import org.apache.tajo.storage.DiskDeviceInfo; -import org.apache.tajo.storage.DiskMountInfo; -import org.apache.tajo.storage.DiskUtil; - -import java.io.File; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.NodeHeartbeat; - -/** - * It periodically sends heartbeat to {@link org.apache.tajo.master.rm.TajoResourceTracker} via asynchronous rpc. - */ -@Deprecated -public class WorkerHeartbeatService extends AbstractService { - /** class logger */ - private final static Log LOG = LogFactory.getLog(WorkerHeartbeatService.class); - - private final TajoWorker.WorkerContext context; - private TajoConf systemConf; - private RpcClientManager connectionManager; - private WorkerHeartbeatThread thread; - private static final float HDFS_DATANODE_STORAGE_SIZE; - - static { - HDFS_DATANODE_STORAGE_SIZE = DiskUtil.getDataNodeStorageSize(); - } - - public WorkerHeartbeatService(TajoWorker.WorkerContext context) { - super(WorkerHeartbeatService.class.getSimpleName()); - this.context = context; - } - - @Override - public void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - this.systemConf = (TajoConf) conf; - - this.connectionManager = RpcClientManager.getInstance(); - thread = new WorkerHeartbeatThread(); - super.serviceInit(conf); - } - - @Override - public void serviceStart() throws Exception { - thread.start(); - super.serviceStart(); - } - - @Override - public void serviceStop() throws Exception { - if(thread.stopped.getAndSet(true)){ - return; - } - - synchronized (thread) { - thread.notifyAll(); - } - - super.serviceStop(); - } - - class WorkerHeartbeatThread extends Thread { - private volatile AtomicBoolean stopped = new AtomicBoolean(false); - ServerStatusProto.System systemInfo; - List diskInfos = Lists.newArrayList(); - float workerDiskSlots; - int workerMemoryMB; - List diskDeviceInfos; - - public WorkerHeartbeatThread() { - int workerCpuCoreNum; - - boolean dedicatedResource = systemConf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DEDICATED); - - try { - diskDeviceInfos = DiskUtil.getDiskDeviceInfos(); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - - if(dedicatedResource) { - float dedicatedMemoryRatio = systemConf.getFloatVar(TajoConf.ConfVars.WORKER_RESOURCE_DEDICATED_MEMORY_RATIO); - int totalMemory = getTotalMemoryMB(); - workerMemoryMB = (int) ((float) (totalMemory) * dedicatedMemoryRatio); - workerCpuCoreNum = Runtime.getRuntime().availableProcessors(); - - if(diskDeviceInfos == null) { - workerDiskSlots = TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.defaultIntVal; - } else { - workerDiskSlots = diskDeviceInfos.size(); - } - } else { - workerMemoryMB = systemConf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); - workerCpuCoreNum = systemConf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); - workerDiskSlots = systemConf.getFloatVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS); - - if (systemConf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && HDFS_DATANODE_STORAGE_SIZE > 0) { - workerDiskSlots = HDFS_DATANODE_STORAGE_SIZE; - } - } - - systemInfo = ServerStatusProto.System.newBuilder() - .setAvailableProcessors(workerCpuCoreNum) - .setFreeMemoryMB(0) - .setMaxMemoryMB(0) - .setTotalMemoryMB(getTotalMemoryMB()) - .build(); - } - - public void run() { - LOG.info("Worker Resource Heartbeat Thread start."); - int sendDiskInfoCount = 0; - - while(!stopped.get()) { - if(sendDiskInfoCount == 0 && diskDeviceInfos != null) { - getDiskUsageInfos(); - } - ServerStatusProto.JvmHeap jvmHeap = - ServerStatusProto.JvmHeap.newBuilder() - .setMaxHeap(Runtime.getRuntime().maxMemory()) - .setFreeHeap(Runtime.getRuntime().freeMemory()) - .setTotalHeap(Runtime.getRuntime().totalMemory()) - .build(); - - ServerStatusProto serverStatus = ServerStatusProto.newBuilder() - .addAllDisk(diskInfos) - .setRunningTaskNum( - context.getTaskRunnerManager() == null ? 1 : context.getTaskRunnerManager().getNumTasks()) - .setSystem(systemInfo) - .setDiskSlots(workerDiskSlots) - .setMemoryResourceMB(workerMemoryMB) - .setJvmHeap(jvmHeap) - .build(); - - NodeHeartbeat heartbeatProto = NodeHeartbeat.newBuilder() - .setConnectionInfo(context.getConnectionInfo().getProto()) - .setServerStatus(serverStatus) - .build(); - - NettyClientBase rmClient = null; - try { - CallFuture callBack = new CallFuture(); - - ServiceTracker serviceTracker = context.getServiceTracker(); - rmClient = connectionManager.getClient(serviceTracker.getResourceTrackerAddress(), - TajoResourceTrackerProtocol.class, true); - TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService resourceTracker = rmClient.getStub(); - resourceTracker.heartbeat(callBack.getController(), heartbeatProto, callBack); - - TajoHeartbeatResponse response = callBack.get(2, TimeUnit.SECONDS); - - QueryCoordinatorProtocol.ClusterResourceSummary clusterResourceSummary = response.getClusterResourceSummary(); - if(clusterResourceSummary.getNumWorkers() > 0) { - context.setNumClusterNodes(clusterResourceSummary.getNumWorkers()); - } - context.setClusterResource(clusterResourceSummary); - - } catch (InterruptedException e) { - break; - } catch (TimeoutException te) { - LOG.warn("Heartbeat response is being delayed.", te); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - - try { - if(!stopped.get()){ - synchronized (thread){ - thread.wait(10 * 1000); - } - } - } catch (InterruptedException e) { - break; - } - sendDiskInfoCount++; - - if(sendDiskInfoCount > 10) { - sendDiskInfoCount = 0; - } - } - - LOG.info("Worker Resource Heartbeat Thread stopped."); - } - - private void getDiskUsageInfos() { - diskInfos.clear(); - for(DiskDeviceInfo eachDevice: diskDeviceInfos) { - List mountInfos = eachDevice.getMountInfos(); - if(mountInfos != null) { - for(DiskMountInfo eachMount: mountInfos) { - File eachFile = new File(eachMount.getMountPath()); - diskInfos.add(ServerStatusProto.Disk.newBuilder() - .setAbsolutePath(eachFile.getAbsolutePath()) - .setTotalSpace(eachFile.getTotalSpace()) - .setFreeSpace(eachFile.getFreeSpace()) - .setUsableSpace(eachFile.getUsableSpace()) - .build()); - } - } - } - } - } - - public static int getTotalMemoryMB() { - javax.management.MBeanServer mBeanServer = java.lang.management.ManagementFactory.getPlatformMBeanServer(); - long max = 0; - Object maxObject = null; - try { - javax.management.ObjectName osName = new javax.management.ObjectName("java.lang:type=OperatingSystem"); - if (!System.getProperty("java.vendor").startsWith("IBM")) { - maxObject = mBeanServer.getAttribute(osName, "TotalPhysicalMemorySize"); - } else { - maxObject = mBeanServer.getAttribute(osName, "TotalPhysicalMemory"); - } - } catch (Throwable t) { - LOG.error(t.getMessage(), t); - } - if (maxObject != null) { - max = ((Long)maxObject).longValue(); - } - return ((int) (max / (1024 * 1024))); - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java index 85d74e2d5a..fa9a32d849 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java @@ -22,14 +22,14 @@ import org.apache.tajo.ipc.TajoWorkerProtocol; public class ExecutionBlockStartEvent extends TaskManagerEvent { - private TajoWorkerProtocol.RunExecutionBlockRequestProto requestProto; + private TajoWorkerProtocol.StartExecutionBlockRequestProto requestProto; - public ExecutionBlockStartEvent(TajoWorkerProtocol.RunExecutionBlockRequestProto requestProto) { + public ExecutionBlockStartEvent(TajoWorkerProtocol.StartExecutionBlockRequestProto requestProto) { super(EventType.EB_START, new ExecutionBlockId(requestProto.getExecutionBlockId())); this.requestProto = requestProto; } - public TajoWorkerProtocol.RunExecutionBlockRequestProto getRequestProto() { + public TajoWorkerProtocol.StartExecutionBlockRequestProto getRequestProto() { return requestProto; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java index 9a1c106865..c0867e3e05 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -31,7 +31,7 @@ public class NodeResourceAllocateEvent extends NodeResourceEvent { public NodeResourceAllocateEvent(BatchAllocationRequestProto request, RpcCallback callback) { - super(EventType.ALLOCATE); + super(EventType.TASK_ALLOCATE); this.callback = callback; this.request = request; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java index 6fd2e0d517..6f6fd9689a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java @@ -24,7 +24,9 @@ public class NodeResourceEvent extends AbstractEvent { - public enum EventType { - START, - STOP - } - - protected final ExecutionBlockId executionBlockId; - - public TaskRunnerEvent(EventType eventType, - ExecutionBlockId executionBlockId) { - super(eventType); - this.executionBlockId = executionBlockId; - } - - public ExecutionBlockId getExecutionBlockId() { - return executionBlockId; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java deleted file mode 100644 index 9406794254..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java +++ /dev/null @@ -1,39 +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.worker.event; - -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.serder.PlanProto; -@Deprecated -public class TaskRunnerStartEvent extends TaskRunnerEvent { - - private final TajoWorkerProtocol.RunExecutionBlockRequestProto request; - - public TaskRunnerStartEvent(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { - super(EventType.START, new ExecutionBlockId(request.getExecutionBlockId())); - this.request = request; - } - - public TajoWorkerProtocol.RunExecutionBlockRequestProto getRequest() { - return request; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java deleted file mode 100644 index 297f30c037..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java +++ /dev/null @@ -1,29 +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.worker.event; - -import org.apache.tajo.ExecutionBlockId; - -@Deprecated -public class TaskRunnerStopEvent extends TaskRunnerEvent { - - public TaskRunnerStopEvent(ExecutionBlockId executionBlockId) { - super(EventType.STOP, executionBlockId); - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java index ff0399cf04..854172c54d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java @@ -22,6 +22,7 @@ import org.apache.tajo.master.rm.WorkerResource; import com.google.gson.annotations.Expose; +import org.apache.tajo.resource.NodeResource; public class WorkerResponse { @@ -35,34 +36,29 @@ public class WorkerResponse { @Expose private int usedMemoryMB; @Expose private int usedCpuCoreSlots; - @Expose private long maxHeap; - @Expose private long freeHeap; - @Expose private long totalHeap; - @Expose private int numRunningTasks; @Expose private int numQueryMasterTasks; @Expose private long lastHeartbeatTime; public WorkerResponse(Worker worker) { - this(worker.getResource()); - + this(worker.getTotalResourceCapability(), worker.getAvailableResource(), + worker.getNumRunningTasks(), worker.getNumRunningQueryMaster()); + this.connectionInfo = new WorkerConnectionInfoResponse(worker.getConnectionInfo()); this.lastHeartbeatTime = worker.getLastHeartbeatTime(); } - - private WorkerResponse(WorkerResource resource) { - this.cpuCoreSlots = resource.getCpuCoreSlots(); - this.memoryMB = resource.getMemoryMB(); - this.usedDiskSlots = resource.getUsedDiskSlots(); - this.usedMemoryMB = resource.getUsedMemoryMB(); - this.usedCpuCoreSlots = resource.getUsedCpuCoreSlots(); - this.maxHeap = resource.getMaxHeap(); - this.freeHeap = resource.getFreeHeap(); - this.totalHeap = resource.getTotalHeap(); - this.numRunningTasks = resource.getNumRunningTasks(); - this.numQueryMasterTasks = resource.getNumQueryMasterTasks(); + + private WorkerResponse(NodeResource total, NodeResource available, int numRunningTasks, int numQueryMasterTasks) { + this.cpuCoreSlots = total.getVirtualCores(); + this.memoryMB = total.getMemory(); + this.diskSlots = total.getDisks(); + this.usedDiskSlots = available.getDisks(); + this.usedMemoryMB = available.getMemory(); + this.usedCpuCoreSlots = available.getVirtualCores(); + this.numRunningTasks = numRunningTasks; + this.numQueryMasterTasks = numQueryMasterTasks; } public WorkerConnectionInfoResponse getConnectionInfo() { @@ -89,18 +85,6 @@ public int getUsedCpuCoreSlots() { return usedCpuCoreSlots; } - public long getMaxHeap() { - return maxHeap; - } - - public long getFreeHeap() { - return freeHeap; - } - - public long getTotalHeap() { - return totalHeap; - } - public int getNumRunningTasks() { return numRunningTasks; } diff --git a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto index 1e25647be5..9be0189be3 100644 --- a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto +++ b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto @@ -32,34 +32,6 @@ import "ContainerProtocol.proto"; package hadoop.yarn; -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; - } - - message JvmHeap { - required int64 maxHeap = 1; - required int64 totalHeap = 2; - required int64 freeHeap = 3; - } - - required System system = 1; - required float diskSlots = 2; - required int32 memoryResourceMB = 3; - repeated Disk disk = 4; - required int32 runningTaskNum = 5; - required JvmHeap jvmHeap = 6; -} - //deplecated message TajoHeartbeat { required WorkerConnectionInfoProto connectionInfo = 1; @@ -76,47 +48,19 @@ message TajoHeartbeatResponse { repeated string params = 2; } required BoolProto heartbeatResult = 1; - required ClusterResourceSummary clusterResourceSummary = 2; optional ResponseCommand responseCommand = 3; } -message ClusterResourceSummary { - required int32 numWorkers = 1; - required int32 totalDiskSlots = 2; - required int32 totalCpuCoreSlots = 3; - required int32 totalMemoryMB = 4; - - required int32 totalAvailableDiskSlots = 5; - required int32 totalAvailableCpuCoreSlots = 6; - required int32 totalAvailableMemoryMB = 7; -} - enum ResourceType { LEAF = 1; INTERMEDIATE = 2; QUERYMASTER = 3; } - -message WorkerResourceReleaseRequest { - required ExecutionBlockIdProto executionBlockId = 1; - repeated TajoContainerIdProto containerIds = 2; +message WorkerConnectionsProto { + repeated WorkerConnectionInfoProto worker = 1; } -message WorkerAllocatedResource { - required TajoContainerIdProto containerId = 1; - required WorkerConnectionInfoProto connectionInfo = 2; - - required int32 allocatedMemoryMB = 3; - required float allocatedDiskSlots = 4; -} - -message WorkerResourceAllocationResponse { - required QueryIdProto queryId = 1; - repeated WorkerAllocatedResource workerAllocatedResource = 2; -} - -/////////////////// message AllocationResourceProto { required int32 workerId = 1; required NodeResourceProto resource = 2; @@ -129,18 +73,18 @@ message NodeResourceRequestProto { required int32 priority = 4; required QueryIdProto queryId = 5; required int32 numContainers = 6; - required NodeResourceProto allocation = 7; - repeated int32 candidateNodes = 8; + required NodeResourceProto capacity = 7; + required int32 runningTasks = 8; + repeated int32 candidateNodes = 9; } message NodeResourceResponseProto { required QueryIdProto queryId = 1; repeated AllocationResourceProto resource = 2; } -/////////////////// + service QueryCoordinatorProtocolService { rpc heartbeat(TajoHeartbeat) returns (TajoHeartbeatResponse); - rpc allocateNodeResources(NodeResourceRequestProto) returns (NodeResourceResponseProto); - rpc releaseWorkerResource(WorkerResourceReleaseRequest) returns (BoolProto); - rpc getAllWorkerResource(NullProto) returns (NodeResourceProto); + rpc reserveNodeResources(NodeResourceRequestProto) returns (NodeResourceResponseProto); + rpc getAllWorkers(NullProto) returns (WorkerConnectionsProto); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto index 855c2c64f8..22c03e6641 100644 --- a/tajo-core/src/main/proto/QueryMasterProtocol.proto +++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto @@ -28,6 +28,7 @@ import "CatalogProtos.proto"; import "PrimitiveProtos.proto"; import "TajoWorkerProtocol.proto"; import "ContainerProtocol.proto"; +import "QueryCoordinatorProtocol.proto"; package hadoop.yarn; @@ -43,4 +44,5 @@ service QueryMasterProtocolService { //from TajoMaster's QueryJobManager rpc killQuery(QueryIdProto) returns (NullProto); rpc executeQuery(QueryExecutionRequestProto) returns (NullProto); + rpc startQueryMaster(AllocationResourceProto) returns (BoolProto); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto index 84d4c08ab5..c9ef067eed 100644 --- a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto +++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto @@ -33,8 +33,10 @@ message NodeHeartbeatRequestProto { required int32 workerId = 1; optional NodeResourceProto totalResource = 2; optional NodeResourceProto availableResource = 3; - optional WorkerConnectionInfoProto connectionInfo = 4; - optional NodeStatusProto status = 5; + optional int32 runningTasks = 4; + optional int32 runningQueryMasters = 5; + optional WorkerConnectionInfoProto connectionInfo = 6; + optional NodeStatusProto status = 7; } message NodeHeartbeatResponseProto { diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 715b1e681a..9386a4aca8 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -184,16 +184,20 @@ message DataChannelProto { optional string storeType = 10; } -message RunExecutionBlockRequestProto { +message StartExecutionBlockRequestProto { required ExecutionBlockIdProto executionBlockId = 1; required WorkerConnectionInfoProto queryMaster = 2; - required string nodeId = 3; - required string containerId = 4; - optional string queryOutputPath = 5; - required KeyValueSetProto queryContext = 6; - required string planJson = 7; - required ShuffleType shuffleType = 8; + optional string queryOutputPath = 3; + + required KeyValueSetProto queryContext = 4; + required string planJson = 5; + required ShuffleType shuffleType = 6; +} + +message StopExecutionBlockRequestProto { + required ExecutionBlockIdProto executionBlockId = 1; + optional ExecutionBlockListProto child = 2; } message ExecutionBlockListProto { @@ -208,19 +212,20 @@ message TaskAllocationRequestProto { message BatchAllocationRequestProto { required ExecutionBlockIdProto executionBlockId = 1; repeated TaskAllocationRequestProto taskRequest = 2; - optional RunExecutionBlockRequestProto executionBlockRequest = 3; //TODO should be refactored + optional StartExecutionBlockRequestProto executionBlockRequest = 3; } message BatchAllocationResponseProto { - repeated TaskAllocationRequestProto cancellationTask = 2; + repeated TaskAllocationRequestProto cancellationTask = 1; } service TajoWorkerProtocolService { rpc ping (TaskAttemptIdProto) returns (BoolProto); // from QueryMaster(Worker) - rpc startExecutionBlock(RunExecutionBlockRequestProto) returns (BoolProto); - rpc stopExecutionBlock(ExecutionBlockIdProto) returns (BoolProto); + //rpc startExecutionBlock(StartExecutionBlockRequestProto) returns (BoolProto); + rpc allocateTasks(BatchAllocationRequestProto) returns (BatchAllocationResponseProto); + rpc stopExecutionBlock(StopExecutionBlockRequestProto) returns (BoolProto); rpc killTaskAttempt(TaskAttemptIdProto) returns (BoolProto); rpc cleanup(QueryIdProto) returns (BoolProto); rpc cleanupExecutionBlocks(ExecutionBlockListProto) returns (BoolProto); diff --git a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp index 3e3c2c2471..92587188c8 100644 --- a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp @@ -33,6 +33,7 @@ <%@ page import="org.apache.tajo.util.history.HistoryReader" %> <%@ page import="org.apache.tajo.util.*" %> <%@ page import="java.util.*" %> +<%@ page import="org.apache.tajo.TajoProtos" %> <% String paramQueryId = request.getParameter("queryId"); @@ -60,13 +61,13 @@ } TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); - List allWorkers = tajoWorker.getWorkerContext() + List allWorkers = tajoWorker.getWorkerContext() .getQueryMasterManagerService().getQueryMaster().getAllWorker(); - Map workerMap = new HashMap(); + Map workerMap = new HashMap(); if(allWorkers != null) { - for(QueryCoordinatorProtocol.WorkerResourceProto eachWorker: allWorkers) { - workerMap.put(eachWorker.getConnectionInfo().getId(), eachWorker); + for(TajoProtos.WorkerConnectionInfoProto eachWorker: allWorkers) { + workerMap.put(eachWorker.getId(), eachWorker); } } QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext() @@ -230,13 +231,13 @@ String taskHost = eachTask.getSucceededHost() == null ? "-" : eachTask.getSucceededHost(); if(eachTask.getSucceededHost() != null) { - QueryCoordinatorProtocol.WorkerResourceProto worker = + TajoProtos.WorkerConnectionInfoProto worker = workerMap.get(eachTask.getLastAttempt().getWorkerConnectionInfo().getId()); if(worker != null) { TaskAttempt lastAttempt = eachTask.getLastAttempt(); if(lastAttempt != null) { TaskAttemptId lastAttemptId = lastAttempt.getId(); - taskHost = "" + eachTask.getSucceededHost() + ""; + taskHost = "" + eachTask.getSucceededHost() + ""; } } } diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index dd54962d8b..b8016f814a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -124,15 +124,8 @@ void initPropertiesAndConfigs() { conf.setClassVar(ConfVars.LOGICAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, LogicalPlanTestRuleProvider.class); conf.setClassVar(ConfVars.GLOBAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, GlobalPlanTestRuleProvider.class); - - // default resource manager - if (System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname) != null) { - String testResourceManager = System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname); - Preconditions.checkState(testResourceManager.equals(TajoResourceManager.class.getCanonicalName())); - conf.set(ConfVars.RESOURCE_MANAGER_CLASS.varname, System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname)); - } - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 2048); - conf.setFloat(ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.varname, 2.0f); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 3072); + conf.setFloat(ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.varname, 4.0f); // Client API RPC @@ -154,7 +147,7 @@ void initPropertiesAndConfigs() { conf.setIntVar(ConfVars.SHUFFLE_RPC_SERVER_WORKER_THREAD_NUM, 2); // Resource allocator - conf.setIntVar(ConfVars.$QUERY_EXECUTE_PARALLEL_MAX, 3); + conf.setIntVar(ConfVars.$QUERY_EXECUTE_PARALLEL_MAX, 10); conf.setIntVar(ConfVars.YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM, 6); // make twice of parallel_max // Memory cache termination diff --git a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java index e5262d8d69..592767ae60 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java @@ -22,10 +22,6 @@ import org.apache.tajo.QueryId; import org.apache.tajo.QueryIdFactory; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.rpc.NullCallback; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.junit.Test; @@ -34,7 +30,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.NodeHeartbeat; import static org.junit.Assert.*; public class TestTajoResourceManager { @@ -47,12 +42,11 @@ public class TestTajoResourceManager { int numWorkers = 5; float workerDiskSlots = 5.0f; int workerMemoryMB = 512 * 10; - WorkerResourceAllocationResponse response; private TajoResourceManager initResourceManager() throws Exception { tajoConf = new org.apache.tajo.conf.TajoConf(); - tajoConf.setFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT, 0.0f); + // tajoConf.setFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT, 0.0f); tajoConf.setIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB, 512); tajoConf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS, "localhost:0"); TajoResourceManager tajoResourceManager = new TajoResourceManager(tajoConf); @@ -60,53 +54,21 @@ private TajoResourceManager initResourceManager() throws Exception { tajoResourceManager.start(); for(int i = 0; i < numWorkers; i++) { - ServerStatusProto.System system = ServerStatusProto.System.newBuilder() - .setAvailableProcessors(1) - .setFreeMemoryMB(workerMemoryMB) - .setMaxMemoryMB(workerMemoryMB) - .setTotalMemoryMB(workerMemoryMB) - .build(); - - ServerStatusProto.JvmHeap jvmHeap = ServerStatusProto.JvmHeap.newBuilder() - .setFreeHeap(workerMemoryMB) - .setMaxHeap(workerMemoryMB) - .setTotalHeap(workerMemoryMB) - .build(); - - ServerStatusProto.Disk disk = ServerStatusProto.Disk.newBuilder() - .setAbsolutePath("/") - .setFreeSpace(0) - .setTotalSpace(0) - .setUsableSpace(0) - .build(); - - List disks = new ArrayList(); - - disks.add(disk); - - ServerStatusProto serverStatus = ServerStatusProto.newBuilder() - .setDiskSlots(workerDiskSlots) - .setMemoryResourceMB(workerMemoryMB) - .setJvmHeap(jvmHeap) - .setSystem(system) - .addAllDisk(disks) - .setRunningTaskNum(0) - .build(); - - WorkerConnectionInfo connectionInfo = - new WorkerConnectionInfo("host" + (i + 1), 28091, 28092, 21000 + i, 28093, 28080); - NodeHeartbeat tajoHeartbeat = NodeHeartbeat.newBuilder() - .setConnectionInfo(connectionInfo.getProto()) - .setServerStatus(serverStatus) - .build(); - tajoResourceManager.getResourceTracker().heartbeat(null, tajoHeartbeat, NullCallback.get()); +// WorkerConnectionInfo connectionInfo = +// new WorkerConnectionInfo("host" + (i + 1), 28091, 28092, 21000 + i, 28093, 28080); +// NodeHeartbeat tajoHeartbeat = NodeHeartbeat.newBuilder() +// .setConnectionInfo(connectionInfo.getProto()) +// .setServerStatus(serverStatus) +// .build(); +// +// tajoResourceManager.getResourceTracker().heartbeat(null, tajoHeartbeat, NullCallback.get()); } return tajoResourceManager; } - + /* @Test public void testHeartbeat() throws Exception { TajoResourceManager tajoResourceManager = null; @@ -114,9 +76,9 @@ public void testHeartbeat() throws Exception { tajoResourceManager = initResourceManager(); assertEquals(numWorkers, tajoResourceManager.getWorkers().size()); for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); - assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); +// WorkerResource resource = worker.getResource(); +// assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); +// assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); } } finally { if (tajoResourceManager != null) { @@ -450,5 +412,5 @@ public void run(WorkerResourceAllocationResponse response) { } } } - + */ } diff --git a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java index 874461fb9d..db769f81d1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java @@ -36,80 +36,80 @@ import static org.junit.Assert.*; public class TestFifoScheduler { - private static TajoTestingCluster cluster; - private static TajoConf conf; - private static TajoClient client; - private static String query = - "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey"; - - @BeforeClass - public static void setUp() throws Exception { - cluster = new TajoTestingCluster(); - cluster.startMiniClusterInLocal(1); - conf = cluster.getConfiguration(); - client = cluster.newTajoClient(); - File file = TPCH.getDataFile("lineitem"); - client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) " - + "using text location 'file://" + file.getAbsolutePath() + "'"); - assertTrue(client.existTable("default.lineitem")); - } - - @AfterClass - public static void tearDown() throws Exception { - if (client != null) client.close(); - if (cluster != null) cluster.shutdownMiniCluster(); - } - - @Test - public final void testKillScheduledQuery() throws Exception { - ClientProtos.SubmitQueryResponse res = client.executeQuery(query); - ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query); - QueryId queryId = new QueryId(res.getQueryId()); - QueryId queryId2 = new QueryId(res2.getQueryId()); - - cluster.waitForQuerySubmitted(queryId); - client.killQuery(queryId2); - assertEquals(TajoProtos.QueryState.QUERY_KILLED, client.getQueryStatus(queryId2).getState()); - } - - @Test - public final void testForwardedQuery() throws Exception { - ClientProtos.SubmitQueryResponse res = client.executeQuery(query); - ClientProtos.SubmitQueryResponse res2 = client.executeQuery("select * from lineitem limit 1"); - assertTrue(res.getIsForwarded()); - assertFalse(res2.getIsForwarded()); - - QueryId queryId = new QueryId(res.getQueryId()); - QueryId queryId2 = new QueryId(res2.getQueryId()); - cluster.waitForQuerySubmitted(queryId); - - assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, client.getQueryStatus(queryId2).getState()); - ResultSet resSet = TajoClientUtil.createResultSet(client, res2, 1); - assertNotNull(resSet); - } - - @Test - public final void testScheduledQuery() throws Exception { - ClientProtos.SubmitQueryResponse res = client.executeQuery("select sleep(1) from lineitem"); - ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query); - ClientProtos.SubmitQueryResponse res3 = client.executeQuery(query); - ClientProtos.SubmitQueryResponse res4 = client.executeQuery(query); - - QueryId queryId = new QueryId(res.getQueryId()); - QueryId queryId2 = new QueryId(res2.getQueryId()); - QueryId queryId3 = new QueryId(res3.getQueryId()); - QueryId queryId4 = new QueryId(res4.getQueryId()); - - cluster.waitForQuerySubmitted(queryId); - - assertFalse(TajoClientUtil.isQueryComplete(client.getQueryStatus(queryId).getState())); - - assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId2).getState()); - assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId3).getState()); - assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId4).getState()); - - client.killQuery(queryId4); - client.killQuery(queryId3); - client.killQuery(queryId2); - } +// private static TajoTestingCluster cluster; +// private static TajoConf conf; +// private static TajoClient client; +// private static String query = +// "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey"; +// +// @BeforeClass +// public static void setUp() throws Exception { +// cluster = new TajoTestingCluster(); +// cluster.startMiniClusterInLocal(1); +// conf = cluster.getConfiguration(); +// client = cluster.newTajoClient(); +// File file = TPCH.getDataFile("lineitem"); +// client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) " +// + "using text location 'file://" + file.getAbsolutePath() + "'"); +// assertTrue(client.existTable("default.lineitem")); +// } +// +// @AfterClass +// public static void tearDown() throws Exception { +// if (client != null) client.close(); +// if (cluster != null) cluster.shutdownMiniCluster(); +// } +// +// @Test +// public final void testKillScheduledQuery() throws Exception { +// ClientProtos.SubmitQueryResponse res = client.executeQuery(query); +// ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query); +// QueryId queryId = new QueryId(res.getQueryId()); +// QueryId queryId2 = new QueryId(res2.getQueryId()); +// +// cluster.waitForQuerySubmitted(queryId); +// client.killQuery(queryId2); +// assertEquals(TajoProtos.QueryState.QUERY_KILLED, client.getQueryStatus(queryId2).getState()); +// } +// +// @Test +// public final void testForwardedQuery() throws Exception { +// ClientProtos.SubmitQueryResponse res = client.executeQuery(query); +// ClientProtos.SubmitQueryResponse res2 = client.executeQuery("select * from lineitem limit 1"); +// assertTrue(res.getIsForwarded()); +// assertFalse(res2.getIsForwarded()); +// +// QueryId queryId = new QueryId(res.getQueryId()); +// QueryId queryId2 = new QueryId(res2.getQueryId()); +// cluster.waitForQuerySubmitted(queryId); +// +// assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, client.getQueryStatus(queryId2).getState()); +// ResultSet resSet = TajoClientUtil.createResultSet(client, res2, 1); +// assertNotNull(resSet); +// } +// +// @Test +// public final void testScheduledQuery() throws Exception { +// ClientProtos.SubmitQueryResponse res = client.executeQuery("select sleep(1) from lineitem"); +// ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query); +// ClientProtos.SubmitQueryResponse res3 = client.executeQuery(query); +// ClientProtos.SubmitQueryResponse res4 = client.executeQuery(query); +// +// QueryId queryId = new QueryId(res.getQueryId()); +// QueryId queryId2 = new QueryId(res2.getQueryId()); +// QueryId queryId3 = new QueryId(res3.getQueryId()); +// QueryId queryId4 = new QueryId(res4.getQueryId()); +// +// cluster.waitForQuerySubmitted(queryId); +// +// assertFalse(TajoClientUtil.isQueryComplete(client.getQueryStatus(queryId).getState())); +// +// assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId2).getState()); +// assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId3).getState()); +// assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId4).getState()); +// +// client.killQuery(queryId4); +// client.killQuery(queryId3); +// client.killQuery(queryId2); +// } } diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index 0cec3dac49..ae32eb4189 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.tajo.*; @@ -49,10 +50,7 @@ import org.apache.tajo.util.history.HistoryReader; import org.apache.tajo.util.history.HistoryWriter; import org.apache.tajo.util.metrics.TajoSystemMetrics; -import org.apache.tajo.worker.ExecutionBlockContext; -import org.apache.tajo.worker.LegacyTaskImpl; -import org.apache.tajo.worker.TajoWorker; -import org.apache.tajo.worker.TaskRunnerManager; +import org.apache.tajo.worker.*; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -243,19 +241,17 @@ public void testKillTask() throws Throwable { final TajoConf conf = new TajoConf(); TaskRequestImpl taskRequest = new TaskRequestImpl(); - taskRequest.set(null, new ArrayList(), + TaskAttemptId attemptId = new TaskAttemptId(tid, 1); + taskRequest.set(attemptId, new ArrayList(), null, false, PlanProto.LogicalNodeTree.newBuilder().build(), new QueryContext(conf), null, null); taskRequest.setInterQuery(); - TaskAttemptId attemptId = new TaskAttemptId(tid, 1); WorkerConnectionInfo queryMaster = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder - requestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder + requestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); requestProto.setExecutionBlockId(eid.getProto()) .setQueryMaster(queryMaster.getProto()) - .setNodeId(queryMaster.getHost()+":" + queryMaster.getQueryMasterPort()) - .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); @@ -265,13 +261,32 @@ public void testKillTask() throws Throwable { public TajoConf getConf() { return conf; } + + @Override + public TaskManager getTaskManager() { + return null; + } + + @Override + public TaskExecutor getTaskExecuor() { + return null; + } + + @Override + public NodeResourceManager getNodeResourceManager() { + return null; + } }; ExecutionBlockContext context = - new ExecutionBlockContext(workerContext, null, requestProto.build()); - - org.apache.tajo.worker.Task task = new LegacyTaskImpl("test", CommonTestingUtil.getTestDir(), attemptId, - conf, context, taskRequest); + new ExecutionBlockContext(workerContext, requestProto.build()) { + @Override + public Path createBaseDir() throws IOException { + return new Path("test"); + } + }; + + org.apache.tajo.worker.Task task = new TaskImpl(taskRequest, context, null); task.kill(); assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); try { @@ -300,94 +315,4 @@ protected void dispatch(Event event) { super.dispatch(event); } } - - abstract class MockWorkerContext implements TajoWorker.WorkerContext { - - @Override - public QueryMaster getQueryMaster() { - return null; - } - - public abstract TajoConf getConf(); - - @Override - public ServiceTracker getServiceTracker() { - return null; - } - - @Override - public QueryMasterManagerService getQueryMasterManagerService() { - return null; - } - - @Override - public TaskRunnerManager getTaskRunnerManager() { - return null; - } - - @Override - public CatalogService getCatalog() { - return null; - } - - @Override - public WorkerConnectionInfo getConnectionInfo() { - return null; - } - - @Override - public String getWorkerName() { - return null; - } - - @Override - public LocalDirAllocator getLocalDirAllocator() { - return null; - } - - @Override - public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { - return null; - } - - @Override - public TajoSystemMetrics getWorkerSystemMetrics() { - return null; - } - - @Override - public HashShuffleAppenderManager getHashShuffleAppenderManager() { - return null; - } - - @Override - public HistoryWriter getTaskHistoryWriter() { - return null; - } - - @Override - public HistoryReader getHistoryReader() { - return null; - } - - @Override - public void cleanup(String strPath) { - - } - - @Override - public void cleanupTemporalDirectories() { - - } - - @Override - public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { - - } - - @Override - public void setNumClusterNodes(int numClusterNodes) { - - } - } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java index 9d4e1f3392..1ca7b92a90 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java @@ -26,8 +26,8 @@ public class MockExecutionBlock extends ExecutionBlockContext { public MockExecutionBlock(TajoWorker.WorkerContext workerContext, - TajoWorkerProtocol.RunExecutionBlockRequestProto request) throws IOException { - super(workerContext, null, request); + TajoWorkerProtocol.StartExecutionBlockRequestProto request) throws IOException { + super(workerContext, request); } @Override diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java index 18b9405c8a..b547916ba6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java @@ -38,8 +38,8 @@ public class MockNodeResourceManager extends NodeResourceManager { volatile boolean enableTaskHandlerEvent = true; private final Semaphore barrier; - public MockNodeResourceManager(Semaphore barrier, Dispatcher dispatcher, EventHandler taskEventHandler) { - super(dispatcher, taskEventHandler); + public MockNodeResourceManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.WorkerContext workerContext) { + super(dispatcher, workerContext); this.barrier = barrier; } @@ -50,7 +50,7 @@ public void handle(NodeResourceEvent event) { } @Override - protected void startExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + protected void startExecutionBlock(TajoWorkerProtocol.StartExecutionBlockRequestProto request) { if(enableTaskHandlerEvent) { super.startExecutionBlock(request); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java index dfcfd4f435..80c1604202 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java @@ -39,9 +39,8 @@ public class MockNodeStatusUpdater extends NodeStatusUpdater { private Map resources = Maps.newHashMap(); private MockResourceTracker resourceTracker; - public MockNodeStatusUpdater(CountDownLatch barrier, TajoWorker.WorkerContext workerContext, - NodeResourceManager resourceManager) { - super(workerContext, resourceManager); + public MockNodeStatusUpdater(CountDownLatch barrier, TajoWorker.WorkerContext workerContext) { + super(workerContext); this.barrier = barrier; this.resourceTracker = new MockResourceTracker(); } @@ -72,12 +71,6 @@ protected NodeHeartbeatRequestProto getLastRequest() { return lastRequest; } - @Override - public void heartbeat(RpcController controller, NodeHeartbeat request, - RpcCallback done) { - - } - @Override public void nodeHeartbeat(RpcController controller, NodeHeartbeatRequestProto request, RpcCallback done) { diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java index f62733f1a2..76de201492 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -32,8 +32,8 @@ public class MockTaskExecutor extends TaskExecutor { protected final Semaphore barrier; - public MockTaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { - super(taskManager, rmEventHandler); + public MockTaskExecutor(Semaphore barrier, TajoWorker.WorkerContext workerContext) { + super(workerContext); this.barrier = barrier; } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java index 678b0636c4..ea82175db3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java @@ -31,13 +31,13 @@ public class MockTaskManager extends TaskManager { private final Semaphore barrier; - public MockTaskManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { - super(dispatcher, workerContext, rmEventHandler); + public MockTaskManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.WorkerContext workerContext) { + super(dispatcher, workerContext); this.barrier = barrier; } @Override - protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.StartExecutionBlockRequestProto request) { try { return new MockExecutionBlock(getWorkerContext(), request); } catch (IOException e) { diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java index e8c2b9c369..f57fa85b72 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java @@ -51,11 +51,6 @@ public QueryMasterManagerService getQueryMasterManagerService() { return null; } - @Override - public TaskRunnerManager getTaskRunnerManager() { - return null; - } - @Override public CatalogService getCatalog() { return null; @@ -76,11 +71,6 @@ public LocalDirAllocator getLocalDirAllocator() { return null; } - @Override - public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { - return null; - } - @Override public TajoSystemMetrics getWorkerSystemMetrics() { @@ -115,15 +105,5 @@ public void cleanup(String strPath) { public void cleanupTemporalDirectories() { } - - @Override - public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { - - } - - @Override - public void setNumClusterNodes(int numClusterNodes) { - - } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java index 65627c1767..a91fc30513 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java @@ -104,13 +104,13 @@ public void testGet() throws IOException { @Test public void testAdjustFetchProcess() { - assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(0, 0), 0); - assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(10, 10), 0); - assertEquals(0.05f, LegacyTaskImpl.adjustFetchProcess(10, 9), 0); - assertEquals(0.1f, LegacyTaskImpl.adjustFetchProcess(10, 8), 0); - assertEquals(0.25f, LegacyTaskImpl.adjustFetchProcess(10, 5), 0); - assertEquals(0.45f, LegacyTaskImpl.adjustFetchProcess(10, 1), 0); - assertEquals(0.5f, LegacyTaskImpl.adjustFetchProcess(10, 0), 0); + assertEquals(0.0f, TaskImpl.adjustFetchProcess(0, 0), 0); + assertEquals(0.0f, TaskImpl.adjustFetchProcess(10, 10), 0); + assertEquals(0.05f, TaskImpl.adjustFetchProcess(10, 9), 0); + assertEquals(0.1f, TaskImpl.adjustFetchProcess(10, 8), 0); + assertEquals(0.25f, TaskImpl.adjustFetchProcess(10, 5), 0); + assertEquals(0.45f, TaskImpl.adjustFetchProcess(10, 1), 0); + assertEquals(0.5f, TaskImpl.adjustFetchProcess(10, 0), 0); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java index df6d714732..dac8bdce2a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java @@ -66,7 +66,7 @@ public static void tearDown() throws IOException { @Test public final void testTaskRunnerHistory() throws IOException, ServiceException, InterruptedException { - int beforeFinishedQueriesCount = master.getContext().getQueryJobManager().getFinishedQueries().size(); + /* int beforeFinishedQueriesCount = master.getContext().getQueryJobManager().getFinishedQueries().size(); client.executeQueryAndGetResult("select count(*) from lineitem"); Collection finishedQueries = master.getContext().getQueryJobManager().getFinishedQueries(); @@ -89,11 +89,12 @@ public final void testTaskRunnerHistory() throws IOException, ServiceException, assertEquals(history.getState(), fromProto.getState()); assertEquals(history.getContainerId(), fromProto.getContainerId()); assertEquals(history.getProto().getTaskHistoriesCount(), fromProto.getProto().getTaskHistoriesCount()); + */ } @Test public final void testTaskHistory() throws IOException, ServiceException, InterruptedException { - int beforeFinishedQueriesCount = master.getContext().getQueryJobManager().getFinishedQueries().size(); + /*int beforeFinishedQueriesCount = master.getContext().getQueryJobManager().getFinishedQueries().size(); client.executeQueryAndGetResult("select count(*) from lineitem"); Collection finishedQueries = master.getContext().getQueryJobManager().getFinishedQueries(); @@ -119,6 +120,6 @@ public final void testTaskHistory() throws IOException, ServiceException, Interr TaskHistory taskHistory = entry.getValue(); assertEquals(TajoProtos.TaskAttemptState.TA_SUCCEEDED, taskHistory.getState()); - assertEquals(taskAttemptId, taskHistory.getTaskAttemptId()); + assertEquals(taskAttemptId, taskHistory.getTaskAttemptId());*/ } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 2cee7d025f..9a6a91d8bd 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -78,6 +78,21 @@ public TajoConf getConf() { return conf; } + @Override + public TaskManager getTaskManager() { + return taskManager; + } + + @Override + public TaskExecutor getTaskExecuor() { + return taskExecutor; + } + + @Override + public NodeResourceManager getNodeResourceManager() { + return resourceManager; + } + @Override public WorkerConnectionInfo getConnectionInfo() { if (workerConnectionInfo == null) { @@ -87,10 +102,10 @@ public WorkerConnectionInfo getConnectionInfo() { } }; - taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); - taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); - resourceManager = new MockNodeResourceManager(new Semaphore(0), dispatcher, taskDispatcher.getEventHandler()); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext); + taskExecutor = new MockTaskExecutor(new Semaphore(0), workerContext); + resourceManager = new MockNodeResourceManager(new Semaphore(0), dispatcher, workerContext); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext); service = new CompositeService("MockService") { @Override @@ -199,13 +214,10 @@ public void testParallelRequest() throws Exception { totalTasks = MockNodeResourceManager.createTaskRequests(ebId, taskMemory, taskSize); // first request with starting ExecutionBlock - TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); ebRequestProto.setExecutionBlockId(ebId.getProto()) .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + - workerContext.getConnectionInfo().getQueryMasterPort()) - .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index af40554f31..450b0aa283 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -18,6 +18,8 @@ package org.apache.tajo.worker; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; @@ -30,13 +32,18 @@ import org.junit.Test; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; + import static org.junit.Assert.*; public class TestNodeStatusUpdater { private NodeResourceManager resourceManager; private MockNodeStatusUpdater statusUpdater; + private MockTaskManager taskManager; private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private CompositeService service; private TajoConf conf; private TajoWorker.WorkerContext workerContext; @@ -53,6 +60,21 @@ public TajoConf getConf() { return conf; } + @Override + public TaskManager getTaskManager() { + return taskManager; + } + + @Override + public TaskExecutor getTaskExecuor() { + return null; + } + + @Override + public NodeResourceManager getNodeResourceManager() { + return resourceManager; + } + @Override public WorkerConnectionInfo getConnectionInfo() { if (workerConnectionInfo == null) { @@ -64,25 +86,40 @@ public WorkerConnectionInfo getConnectionInfo() { conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL, 1000); dispatcher = new AsyncDispatcher(); - dispatcher.init(conf); - dispatcher.start(); + resourceManager = new NodeResourceManager(dispatcher, workerContext); + taskDispatcher = new AsyncDispatcher(); + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext) { + @Override + public int getRunningTasks() { + return 0; + } + }; + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + }; - resourceManager = new NodeResourceManager(dispatcher, null); - resourceManager.init(conf); - resourceManager.start(); + service.init(conf); + service.start(); } @After public void tearDown() { - resourceManager.stop(); - if (statusUpdater != null) statusUpdater.stop(); - dispatcher.stop(); + service.stop(); } @Test(timeout = 20000) public void testNodeMembership() throws Exception { CountDownLatch barrier = new CountDownLatch(1); - statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext); statusUpdater.init(conf); statusUpdater.start(); @@ -100,7 +137,7 @@ public void testNodeMembership() throws Exception { @Test(timeout = 20000) public void testPing() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext); statusUpdater.init(conf); statusUpdater.start(); @@ -117,7 +154,7 @@ public void testPing() throws Exception { @Test(timeout = 20000) public void testResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext); statusUpdater.init(conf); statusUpdater.start(); @@ -132,7 +169,7 @@ public void testResourceReport() throws Exception { @Test(timeout = 20000) public void testFlushResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext); statusUpdater.init(conf); statusUpdater.start(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index 98b187b013..72b8f64a57 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -92,6 +92,21 @@ public TajoConf getConf() { return conf; } + @Override + public TaskManager getTaskManager() { + return taskManager; + } + + @Override + public org.apache.tajo.worker.TaskExecutor getTaskExecuor() { + return taskExecutor; + } + + @Override + public NodeResourceManager getNodeResourceManager() { + return resourceManager; + } + @Override public WorkerConnectionInfo getConnectionInfo() { if (workerConnectionInfo == null) { @@ -103,10 +118,10 @@ public WorkerConnectionInfo getConnectionInfo() { barrier = new Semaphore(0); resourceManagerBarrier = new Semaphore(0); - taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); - taskExecutor = new TaskExecutor(barrier, taskManager, dispatcher.getEventHandler()); - resourceManager = new MockNodeResourceManager(resourceManagerBarrier, dispatcher, taskDispatcher.getEventHandler()); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext); + taskExecutor = new TaskExecutor(barrier, workerContext); + resourceManager = new MockNodeResourceManager(resourceManagerBarrier, dispatcher, workerContext); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext); service = new CompositeService("MockService") { @Override @@ -141,16 +156,13 @@ public void tearDown() { public void testTaskRequest() throws Exception { int requestSize = 1; - RunExecutionBlockRequestProto.Builder - ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + StartExecutionBlockRequestProto.Builder + ebRequestProto = StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setNodeId(workerContext.getConnectionInfo().getHost() + ":" - + workerContext.getConnectionInfo().getQueryMasterPort()) - .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); @@ -181,16 +193,13 @@ public void testTaskRequest() throws Exception { public void testTaskException() throws Exception { int requestSize = 1; - RunExecutionBlockRequestProto.Builder - ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + StartExecutionBlockRequestProto.Builder + ebRequestProto = StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setNodeId(workerContext.getConnectionInfo().getHost()+":" - + workerContext.getConnectionInfo().getQueryMasterPort()) - .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); @@ -222,8 +231,8 @@ class TaskExecutor extends MockTaskExecutor { int completeTasks; AtomicBoolean throwException = new AtomicBoolean(); - public TaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { - super(barrier, taskManager, rmEventHandler); + public TaskExecutor(Semaphore barrier, TajoWorker.WorkerContext workerContext) { + super(barrier, workerContext); } @Override diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index 8bca489e20..cbee793693 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -79,6 +79,21 @@ public TajoConf getConf() { return conf; } + @Override + public TaskManager getTaskManager() { + return taskManager; + } + + @Override + public TaskExecutor getTaskExecuor() { + return taskExecutor; + } + + @Override + public NodeResourceManager getNodeResourceManager() { + return resourceManager; + } + @Override public WorkerConnectionInfo getConnectionInfo() { if (workerConnectionInfo == null) { @@ -88,10 +103,10 @@ public WorkerConnectionInfo getConnectionInfo() { } }; barrier = new Semaphore(0); - taskManager = new MockTaskManager(barrier, taskDispatcher, workerContext, dispatcher.getEventHandler()); - taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); - resourceManager = new NodeResourceManager(dispatcher, taskDispatcher.getEventHandler()); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); + taskManager = new MockTaskManager(barrier, taskDispatcher, workerContext); + taskExecutor = new MockTaskExecutor(new Semaphore(0), workerContext); + resourceManager = new NodeResourceManager(dispatcher, workerContext); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext); service = new CompositeService("MockService") { @Override @@ -126,16 +141,13 @@ public void tearDown() { public void testExecutionBlockStart() throws Exception { int requestSize = 1; - TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setNodeId(workerContext.getConnectionInfo().getHost() + ":" - + workerContext.getConnectionInfo().getQueryMasterPort()) - .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); @@ -158,16 +170,13 @@ public void testExecutionBlockStart() throws Exception { @Test(timeout = 10000) public void testExecutionBlockStop() throws Exception { - TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setNodeId(workerContext.getConnectionInfo().getHost()+":" - + workerContext.getConnectionInfo().getQueryMasterPort()) - .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); From 133e3f2c04a435a6da9ae823be0fa2dd803db377 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 18 Jun 2015 00:50:18 +0900 Subject: [PATCH 04/80] TAJO-1397 --- .../apache/tajo/catalog/CatalogServer.java | 16 +- .../org/apache/tajo/cli/tools/TajoAdmin.java | 23 +- tajo-client/src/main/proto/ClientProtos.proto | 19 +- .../java/org/apache/tajo/conf/TajoConf.java | 32 +- .../apache/tajo/master/ContainerProxy.java | 85 ---- .../tajo/master/LaunchTaskRunnersEvent.java | 46 -- .../tajo/master/QueryCoordinatorService.java | 21 +- .../apache/tajo/master/QueryInProgress.java | 32 +- .../org/apache/tajo/master/QueryManager.java | 46 +- .../tajo/master/TajoContainerProxy.java | 177 ------- .../org/apache/tajo/master/TajoMaster.java | 110 ++-- .../tajo/master/TajoMasterClientService.java | 64 +-- .../tajo/master/TaskRunnerGroupEvent.java | 51 -- .../event/ContainerAllocationEvent.java | 77 --- .../tajo/master/event/ContainerEvent.java | 37 -- .../GrouppedContainerAllocatorEvent.java | 45 -- .../tajo/master/event/LocalTaskEvent.java | 10 +- .../tajo/master/event/QueryStartEvent.java | 9 +- .../tajo/master/event/StageEventType.java | 2 - .../tajo/master/event/TaskRequestEvent.java | 4 - .../NonForwardQueryResultSystemScanner.java | 20 +- .../apache/tajo/master/rm/TajoRMContext.java | 20 - .../tajo/master/rm/TajoResourceManager.java | 11 +- .../tajo/master/rm/TajoResourceTracker.java | 21 +- .../org/apache/tajo/master/rm/Worker.java | 7 +- .../tajo/master/rm/WorkerResourceManager.java | 110 ---- .../tajo/master/rm/WorkerStatusEvent.java | 12 +- .../scheduler/AbstractQueryScheduler.java | 66 +++ .../master/scheduler/QuerySchedulingInfo.java | 4 + .../tajo/master/scheduler/QueueInfo.java | 101 ++++ .../QueueState.java} | 48 +- .../master/scheduler/SimpleScheduler.java | 474 ++++++++++++++++++ .../scheduler/TajoResourceScheduler.java | 68 +++ .../event/ResourceReserveSchedulerEvent.java | 45 ++ .../event/SchedulerEvent.java} | 13 +- .../event/SchedulerEventType.java} | 12 +- .../querymaster/DefaultTaskScheduler.java | 5 - .../apache/tajo/querymaster/QueryMaster.java | 177 +++---- .../QueryMasterManagerService.java | 86 +--- .../tajo/querymaster/QueryMasterTask.java | 168 ++++--- .../org/apache/tajo/querymaster/Stage.java | 220 ++------ .../apache/tajo/querymaster/TaskAttempt.java | 3 +- .../worker/AbstractResourceAllocator.java | 69 --- .../tajo/worker/NodeResourceManager.java | 15 +- .../apache/tajo/worker/NodeStatusUpdater.java | 23 +- .../tajo/worker/TajoResourceAllocator.java | 413 --------------- .../org/apache/tajo/worker/TajoWorker.java | 6 +- .../tajo/worker/TajoWorkerManagerService.java | 53 +- .../org/apache/tajo/worker/TaskExecutor.java | 15 +- .../org/apache/tajo/worker/TaskManager.java | 6 +- .../worker/event/QMResourceAllocateEvent.java | 50 ++ .../src/main/proto/QueryMasterProtocol.proto | 3 +- .../src/main/proto/TajoWorkerProtocol.proto | 15 +- .../org/apache/tajo/TajoTestingCluster.java | 3 +- .../master/rm/TestTajoResourceManager.java | 2 +- .../master/scheduler/TestFifoScheduler.java | 115 ----- .../tajo/querymaster/TestKillQuery.java | 21 +- .../tajo/worker/TestNodeResourceManager.java | 2 +- .../tajo/worker/TestNodeStatusUpdater.java | 3 +- .../apache/tajo/worker/TestTaskManager.java | 2 +- 60 files changed, 1338 insertions(+), 2075 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/master/scheduler/QueueInfo.java rename tajo-core/src/main/java/org/apache/tajo/master/{event/StageContainerAllocationEvent.java => scheduler/QueueState.java} (50%) create mode 100644 tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java rename tajo-core/src/main/java/org/apache/tajo/master/{event/ContainerAllocatorEventType.java => scheduler/event/SchedulerEvent.java} (76%) rename tajo-core/src/main/java/org/apache/tajo/master/{TaskRunnerLauncher.java => scheduler/event/SchedulerEventType.java} (81%) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.java delete mode 100644 tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.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 f2e9795537..b1410dd0c4 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 @@ -172,7 +172,8 @@ private void initBuiltinFunctions(Collection functions) } } - public void start() { + @Override + public void serviceStart() throws Exception { String serverAddr = conf.getVar(ConfVars.CATALOG_ADDRESS); InetSocketAddress initIsa = NetUtils.createSocketAddr(serverAddr); int workerNum = conf.getIntVar(ConfVars.CATALOG_RPC_SERVER_WORKER_THREAD_NUM); @@ -189,10 +190,11 @@ public void start() { } LOG.info("Catalog Server startup (" + bindAddressStr + ")"); - super.start(); + super.serviceStart(); } - public void stop() { + @Override + public void serviceStop() throws Exception { LOG.info("Catalog Server (" + bindAddressStr + ") shutdown"); // If CatalogServer shutdowns before it started, rpcServer and store may be NULL. @@ -201,13 +203,9 @@ public void stop() { this.rpcServer.shutdown(); } if (store != null) { - try { - store.close(); - } catch (IOException ioe) { - LOG.error(ioe.getMessage(), ioe); - } + store.close(); } - super.stop(); + super.serviceStop(); } public CatalogProtocolHandler getHandler() { diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java index 739cd541d0..76ba7a9dad 100644 --- a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java +++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java @@ -268,7 +268,7 @@ private void processCluster(Writer writer) throws ParseException, IOException, } else { String fmtQueryMasterLine = "%1$-25s %2$-5s %3$-5s %4$-10s %5$-10s%n"; line = String.format(fmtQueryMasterLine, "QueryMaster", "Port", "Query", - "Heap", "Status"); + "Mem", "Status"); writer.write(line); line = String.format(fmtQueryMasterLine, DASHLINE_LEN25, DASHLINE_LEN5, DASHLINE_LEN5, DASHLINE_LEN10, DASHLINE_LEN10); @@ -276,12 +276,12 @@ private void processCluster(Writer writer) throws ParseException, IOException, for (WorkerResourceInfo queryMaster : liveQueryMasters) { TajoProtos.WorkerConnectionInfoProto connInfo = queryMaster.getConnectionInfo(); String queryMasterHost = String.format("%s:%d", connInfo.getHost(), connInfo.getQueryMasterPort()); - String heap = String.format("%d MB", queryMaster.getMaxHeap() / 1024 / 1024); + String memory = String.format("%d MB", queryMaster.getAvailableResource().getMemory()); line = String.format(fmtQueryMasterLine, queryMasterHost, connInfo.getClientPort(), queryMaster.getNumQueryMasterTasks(), - heap, + memory, queryMaster.getWorkerStatus()); writer.write(line); } @@ -348,7 +348,7 @@ private void writeWorkerInfo(Writer writer, List workers) th String line = String.format(fmtWorkerLine, "Worker", "Port", "Tasks", "Mem", "Disk", - "Heap", "Status"); + "Cpu", "Status"); writer.write(line); line = String.format(fmtWorkerLine, DASHLINE_LEN25, DASHLINE_LEN5, DASHLINE_LEN5, @@ -359,17 +359,16 @@ private void writeWorkerInfo(Writer writer, List workers) th for (WorkerResourceInfo worker : workers) { TajoProtos.WorkerConnectionInfoProto connInfo = worker.getConnectionInfo(); String workerHost = String.format("%s:%d", connInfo.getHost(), connInfo.getPeerRpcPort()); - String mem = String.format("%d/%d", worker.getUsedMemoryMB(), - worker.getMemoryMB()); - String disk = String.format("%.2f/%.2f", worker.getUsedDiskSlots(), - worker.getDiskSlots()); - String heap = String.format("%d/%d MB", worker.getFreeHeap()/1024/1024, - worker.getMaxHeap()/1024/1024); - + String mem = String.format("%d/%d", worker.getAvailableResource().getMemory(), + worker.getTotalResource().getMemory()); + String disk = String.format("%d/%d", worker.getAvailableResource().getDisks(), + worker.getTotalResource().getDisks()); + String cpu = String.format("%d/%d", worker.getAvailableResource().getVirtualCores(), + worker.getTotalResource().getVirtualCores()); line = String.format(fmtWorkerLine, workerHost, connInfo.getPullServerPort(), worker.getNumRunningTasks(), - mem, disk, heap, worker.getWorkerStatus()); + mem, disk, cpu, worker.getWorkerStatus()); writer.write(line); } writer.write("\n\n"); diff --git a/tajo-client/src/main/proto/ClientProtos.proto b/tajo-client/src/main/proto/ClientProtos.proto index 5497faade5..9c20fd8e3a 100644 --- a/tajo-client/src/main/proto/ClientProtos.proto +++ b/tajo-client/src/main/proto/ClientProtos.proto @@ -177,19 +177,12 @@ message GetClusterInfoRequest { message WorkerResourceInfo { required WorkerConnectionInfoProto connectionInfo = 1; - required float diskSlots = 2; - required int32 cpuCoreSlots = 3; - required int32 memoryMB = 4; - required float usedDiskSlots = 5; - required int32 usedMemoryMB = 6; - required int32 usedCpuCoreSlots = 7; - required int64 maxHeap = 8; - required int64 freeHeap = 9; - required int64 totalHeap = 10; - required int32 numRunningTasks = 11; - required string workerStatus = 12; - required int64 lastHeartbeat = 13; - required int32 numQueryMasterTasks = 14; + required NodeResourceProto totalResource = 2; + required NodeResourceProto availableResource = 3; + required int32 numRunningTasks = 4; + required string workerStatus = 5; + required int64 lastHeartbeat = 6; + required int32 numQueryMasterTasks = 7; } message GetClusterInfoResponse { 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 1aebba7c23..da6a529315 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 @@ -155,7 +155,10 @@ public static enum ConfVars implements ConfigKey { RESOURCE_TRACKER_HEARTBEAT_TIMEOUT("tajo.resource-tracker.heartbeat.timeout-secs", 120 * 1000), // seconds // QueryMaster resource - TAJO_QUERYMASTER_MEMORY_MB("tajo.qm.resource.memory-mb", 512, Validators.min("64")), + TAJO_QUERYMASTER_MINIMUM_MEMORY("tajo.qm.resource.min.memory-mb", 500, Validators.min("64")), + + // Worker task resource + TASK_RESOURCE_MINIMUM_MEMORY("tajo.task.resource.min.memory-mb", 500, Validators.min("64")), // Tajo Worker Service Addresses WORKER_INFO_ADDRESS("tajo.worker.info-http.address", "0.0.0.0:28080", Validators.networkAddr()), @@ -171,19 +174,14 @@ public static enum ConfVars implements ConfigKey { // Tajo Worker Resources WORKER_RESOURCE_AVAILABLE_CPU_CORES("tajo.worker.resource.cpu-cores", Runtime.getRuntime().availableProcessors(), Validators.min("1")), - WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 1024, Validators.min("64")), - @Deprecated - WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1.0f), - WORKER_RESOURCE_AVAILABLE_DISKS_NUM("tajo.worker.resource.disks.num", 1, Validators.min("1")), + WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 1000, Validators.min("64")), + + WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1, Validators.min("1")), + WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM("tajo.worker.resource.disk.parallel-execution.num", 2, Validators.min("1")), - WORKER_EXECUTION_MAX_SLOTS("tajo.worker.parallel-execution.max-num", 2), - WORKER_RESOURCE_DFS_DIR_AWARE("tajo.worker.resource.dfs-dir-aware", false, Validators.bool()), - // Tajo Worker Dedicated Resources - WORKER_RESOURCE_DEDICATED("tajo.worker.resource.dedicated", false, Validators.bool()), - WORKER_RESOURCE_DEDICATED_MEMORY_RATIO("tajo.worker.resource.dedicated-memory-ratio", 0.8f, - Validators.range("0.0f", "1.0f")), + WORKER_RESOURCE_DFS_DIR_AWARE("tajo.worker.resource.dfs-dir-aware", false, Validators.bool()), // Tajo History WORKER_HISTORY_EXPIRE_PERIOD("tajo.worker.history.expire-interval-minutes", 60), // 1 hours @@ -191,21 +189,13 @@ public static enum ConfVars implements ConfigKey { WORKER_HEARTBEAT_INTERVAL("tajo.worker.heartbeat.interval", 10 * 1000), // 10 sec - // Resource Scheduler + //Default query scheduler RESOURCE_SCHEDULER_CLASS("tajo.resource.scheduler", "org.apache.tajo.master.scheduler.SimpleScheduler", Validators.groups(Validators.notNull(), Validators.clazz())), // Catalog CATALOG_ADDRESS("tajo.catalog.client-rpc.address", "localhost:26005", Validators.networkAddr()), - - // for Yarn Resource Manager ---------------------------------------------- - - /** how many launching TaskRunners in parallel */ - @Deprecated - YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM("tajo.yarn-rm.parallel-task-runner-launcher-num", - Runtime.getRuntime().availableProcessors() * 2), - // Query Configuration QUERY_SESSION_TIMEOUT("tajo.query.session.timeout-sec", 60, Validators.min("0")), QUERY_SESSION_QUERY_CACHE_SIZE("tajo.query.session.query-cache-size-kb", 1024, Validators.min("0")), @@ -260,8 +250,6 @@ public static enum ConfVars implements ConfigKey { Runtime.getRuntime().availableProcessors() * 1), // Task Configuration ----------------------------------------------------- - TASK_DEFAULT_MEMORY("tajo.task.memory-slot-mb.default", 512), - TASK_DEFAULT_DISK("tajo.task.disk-slot.default", 0.5f), TASK_DEFAULT_SIZE("tajo.task.size-mb", 128), // Query and Optimization ------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java deleted file mode 100644 index cad63a0d6d..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java +++ /dev/null @@ -1,85 +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.fs.permission.FsPermission; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.querymaster.QueryMasterTask; -import org.apache.tajo.master.container.TajoContainer; -import org.apache.tajo.master.container.TajoContainerId; - -public abstract class ContainerProxy { - protected static final Log LOG = LogFactory.getLog(ContainerProxy.class); - - final public static FsPermission QUERYCONF_FILE_PERMISSION = - FsPermission.createImmutable((short) 0644); // rw-r--r-- - - - protected static enum ContainerState { - PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH - } - - protected final ExecutionBlockId executionBlockId; - protected Configuration conf; - protected QueryMasterTask.QueryMasterTaskContext context; - - protected ContainerState state; - // store enough information to be able to cleanup the container - protected TajoContainer container; - protected TajoContainerId containerId; - protected String hostName; - protected int port = -1; - - public abstract void launch(ContainerLaunchContext containerLaunchContext); - public abstract void stopContainer(); - - public ContainerProxy(QueryMasterTask.QueryMasterTaskContext context, Configuration conf, - ExecutionBlockId executionBlockId, TajoContainer container) { - this.context = context; - this.conf = conf; - this.state = ContainerState.PREP; - this.container = container; - this.executionBlockId = executionBlockId; - this.containerId = container.getId(); - } - - public synchronized boolean isCompletelyDone() { - return state == ContainerState.DONE || state == ContainerState.FAILED; - } - - public String getTaskHostName() { - return this.hostName; - } - - public int getTaskPort() { - return this.port; - } - - public TajoContainerId getContainerId() { - return containerId; - } - - public ExecutionBlockId getBlockId() { - return executionBlockId; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java deleted file mode 100644 index e620afa161..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java +++ /dev/null @@ -1,46 +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.tajo.ExecutionBlockId; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.master.container.TajoContainer; - -import java.util.Collection; - -public class LaunchTaskRunnersEvent extends TaskRunnerGroupEvent { - private final QueryContext queryContext; - private final String planJson; - - public LaunchTaskRunnersEvent(ExecutionBlockId executionBlockId, - Collection containers, QueryContext queryContext, - String planJson) { - super(EventType.CONTAINER_REMOTE_LAUNCH, executionBlockId, containers); - this.queryContext = queryContext; - this.planJson = planJson; - } - - public QueryContext getQueryContext() { - return queryContext; - } - - public String getPlanJson() { - return planJson; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java index 330451dac2..8acc9d901b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java @@ -58,30 +58,27 @@ public QueryCoordinatorService(TajoMaster.MasterContext context) { } @Override - public void start() { + public void serviceStart() throws Exception { String confMasterServiceAddr = conf.getVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS); InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr); int workerNum = conf.getIntVar(TajoConf.ConfVars.MASTER_RPC_SERVER_WORKER_THREAD_NUM); - try { - server = new AsyncRpcServer(QueryCoordinatorProtocol.class, masterHandler, initIsa, workerNum); - } catch (Exception e) { - LOG.error(e, e); - } + + server = new AsyncRpcServer(QueryCoordinatorProtocol.class, masterHandler, initIsa, workerNum); server.start(); bindAddress = NetUtils.getConnectAddress(server.getListenAddress()); this.conf.setVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress)); LOG.info("Instantiated TajoMasterService at " + this.bindAddress); - super.start(); + super.serviceStart(); } @Override - public void stop() { + public void serviceStop() throws Exception { if(server != null) { server.shutdown(); server = null; } - super.stop(); + super.serviceStop(); } public InetSocketAddress getBindAddress() { @@ -96,17 +93,17 @@ private class ProtocolServiceHandler implements QueryCoordinatorProtocolService. @Override public void heartbeat( RpcController controller, - TajoHeartbeat request, RpcCallback done) { + TajoHeartbeat request, RpcCallback done) { if(LOG.isDebugEnabled()) { LOG.debug("Received QueryHeartbeat:" + new WorkerConnectionInfo(request.getConnectionInfo())); } - QueryCoordinatorProtocol.TajoHeartbeatResponse.ResponseCommand command = null; + TajoHeartbeatResponse.ResponseCommand command; QueryManager queryManager = context.getQueryJobManager(); command = queryManager.queryHeartbeat(request); - QueryCoordinatorProtocol.TajoHeartbeatResponse.Builder builder = QueryCoordinatorProtocol.TajoHeartbeatResponse.newBuilder(); + TajoHeartbeatResponse.Builder builder = TajoHeartbeatResponse.newBuilder(); builder.setHeartbeatResult(BOOL_TRUE); if(command != null) { builder.setResponseCommand(command); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java index 1936f28b91..9511587fd6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java @@ -56,7 +56,7 @@ public class QueryInProgress { private volatile boolean querySubmitted = false; - private AtomicBoolean stopped = new AtomicBoolean(false); + private AtomicBoolean isStopped = new AtomicBoolean(false); private QueryInfo queryInfo; @@ -66,8 +66,7 @@ public class QueryInProgress { private QueryMasterProtocolService queryMasterRpcClient; - //FIXME - private QueryCoordinatorProtocol.AllocationResourceProto allocation; + private QueryCoordinatorProtocol.AllocationResourceProto allocationResource; private final Lock readLock; private final Lock writeLock; @@ -108,7 +107,7 @@ public void kill() { } public void stopProgress() { - if(stopped.getAndSet(true)) { + if(isStopped.getAndSet(true)) { return; } @@ -126,7 +125,11 @@ public void stopProgress() { } } - public boolean startQueryMaster(QueryCoordinatorProtocol.AllocationResourceProto allocation) { + /** + * Connect to QueryMaster and allocate QM resource. + * If there is no available resource, It returns false + */ + protected boolean allocateToQueryMaster(QueryCoordinatorProtocol.AllocationResourceProto allocation) { try { writeLock.lockInterruptibly(); } catch (Exception e) { @@ -134,7 +137,6 @@ public boolean startQueryMaster(QueryCoordinatorProtocol.AllocationResourceProto return false; } try { - this.allocation = allocation; TajoResourceManager resourceManager = masterContext.getResourceManager(); WorkerConnectionInfo connectionInfo = resourceManager.getRMContext().getWorkers().get(allocation.getWorkerId()).getConnectionInfo(); @@ -144,7 +146,7 @@ public boolean startQueryMaster(QueryCoordinatorProtocol.AllocationResourceProto } CallFuture callFuture = new CallFuture(); - queryMasterRpcClient.startQueryMaster(callFuture.getController(), allocation, callFuture); + queryMasterRpcClient.allocateQueryMaster(callFuture.getController(), allocation, callFuture); if(!callFuture.get().getValue()) return false; @@ -153,10 +155,11 @@ public boolean startQueryMaster(QueryCoordinatorProtocol.AllocationResourceProto } LOG.info("Initializing QueryInProgress for QueryID=" + queryId); - queryInfo.setQueryMaster(connectionInfo.getHost()); - queryInfo.setQueryMasterPort(connectionInfo.getQueryMasterPort()); - queryInfo.setQueryMasterclientPort(connectionInfo.getClientPort()); - queryInfo.setQueryMasterInfoPort(connectionInfo.getHttpInfoPort()); + this.allocationResource = allocation; + this.queryInfo.setQueryMaster(connectionInfo.getHost()); + this.queryInfo.setQueryMasterPort(connectionInfo.getQueryMasterPort()); + this.queryInfo.setQueryMasterclientPort(connectionInfo.getClientPort()); + this.queryInfo.setQueryMasterInfoPort(connectionInfo.getHttpInfoPort()); return true; } catch (Exception e) { @@ -172,12 +175,12 @@ private void connectQueryMaster(WorkerConnectionInfo connectionInfo) RpcClientManager.cleanup(queryMasterRpc); InetSocketAddress addr = NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getQueryMasterPort()); - LOG.info("Connect to QueryMaster:" + addr); + LOG.info("Try to connect to QueryMaster:" + addr); queryMasterRpc = RpcClientManager.getInstance().newClient(addr, QueryMasterProtocol.class, true); queryMasterRpcClient = queryMasterRpc.getStub(); } - public boolean submitQueryToMaster() { + public boolean submitToQueryMaster() { if(querySubmitted) { return false; } @@ -199,7 +202,8 @@ public boolean submitQueryToMaster() { .setQueryContext(queryInfo.getQueryContext().getProto()) .setSession(session.getProto()) .setExprInJson(PrimitiveProtos.StringProto.newBuilder().setValue(queryInfo.getJsonExpr())) - .setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build()); + .setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build()) + .setAllocation(allocationResource); CallFuture callFuture = new CallFuture(); queryMasterRpcClient.executeQuery(callFuture.getController(), builder.build(), callFuture); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java index 771edf73a4..d2b881f98c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java @@ -89,10 +89,8 @@ public void serviceInit(Configuration conf) throws Exception { @Override public void serviceStop() throws Exception { - synchronized(runningQueries) { - for(QueryInProgress eachQueryInProgress: runningQueries.values()) { - eachQueryInProgress.stopProgress(); - } + for(QueryInProgress eachQueryInProgress: runningQueries.values()) { + eachQueryInProgress.stopProgress(); } super.serviceStop(); @@ -108,15 +106,11 @@ public EventHandler getEventHandler() { } public Collection getSubmittedQueries() { - synchronized (submittedQueries){ - return Collections.unmodifiableCollection(submittedQueries.values()); - } + return Collections.unmodifiableCollection(submittedQueries.values()); } public Collection getRunningQueries() { - synchronized (runningQueries){ - return Collections.unmodifiableCollection(runningQueries.values()); - } + return Collections.unmodifiableCollection(runningQueries.values()); } public synchronized Collection getFinishedQueries() { @@ -177,6 +171,7 @@ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, Strin queryInProgress.getQueryInfo().setQueryMaster(""); submittedQueries.put(queryInProgress.getQueryId(), queryInProgress); + //TODO implement scheduler queue QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo("default", queryContext.getUser(), queryInProgress.getQueryId(), 1, queryInProgress.getQueryInfo().getStartTime()); @@ -184,19 +179,19 @@ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, Strin return queryInProgress.getQueryInfo(); } + /** + * Can start query or not + */ public boolean startQueryJob(QueryId queryId, QueryCoordinatorProtocol.AllocationResourceProto allocation) { - if (submittedQueries.get(queryId).startQueryMaster(allocation)) { + if (submittedQueries.get(queryId).allocateToQueryMaster(allocation)) { QueryInProgress queryInProgress = submittedQueries.remove(queryId); runningQueries.put(queryInProgress.getQueryId(), queryInProgress); dispatcher.getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, queryInProgress.getQueryInfo())); - } else { - //masterContext.getQueryJobManager().stopQuery(queryInProgress.getQueryId()); - return false; + return true; } - - return true; + return false; } class QueryJobManagerEventHandler implements EventHandler { @@ -211,7 +206,7 @@ public void handle(QueryJobEvent event) { } if (event.getType() == QueryJobEvent.Type.QUERY_MASTER_START) { - queryInProgress.submitQueryToMaster(); + queryInProgress.submitToQueryMaster(); } else if (event.getType() == QueryJobEvent.Type.QUERY_JOB_KILL) { @@ -226,14 +221,10 @@ public void handle(QueryJobEvent event) { public QueryInProgress getQueryInProgress(QueryId queryId) { QueryInProgress queryInProgress; - synchronized (submittedQueries) { - queryInProgress = submittedQueries.get(queryId); - } + queryInProgress = submittedQueries.get(queryId); if (queryInProgress == null) { - synchronized (runningQueries) { - queryInProgress = runningQueries.get(queryId); - } + queryInProgress = runningQueries.get(queryId); } return queryInProgress; } @@ -243,13 +234,8 @@ public void stopQuery(QueryId queryId) { QueryInProgress queryInProgress = getQueryInProgress(queryId); if(queryInProgress != null) { queryInProgress.stopProgress(); - synchronized(submittedQueries) { - submittedQueries.remove(queryId); - } - - synchronized(runningQueries) { - runningQueries.remove(queryId); - } + submittedQueries.remove(queryId); + runningQueries.remove(queryId); QueryInfo queryInfo = queryInProgress.getQueryInfo(); synchronized (historyCache) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java deleted file mode 100644 index d66674dbc2..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java +++ /dev/null @@ -1,177 +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.hadoop.conf.Configuration; -import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.container.TajoContainer; -import org.apache.tajo.master.container.TajoContainerId; -import org.apache.tajo.master.event.TaskFatalErrorEvent; -import org.apache.tajo.master.rm.TajoWorkerContainer; -import org.apache.tajo.master.rm.TajoWorkerContainerId; -import org.apache.tajo.plan.serder.PlanProto; -import org.apache.tajo.querymaster.QueryMasterTask; -import org.apache.tajo.rpc.NettyClientBase; -import org.apache.tajo.rpc.NullCallback; -import org.apache.tajo.rpc.RpcClientManager; -import org.apache.tajo.service.ServiceTracker; -import org.apache.tajo.worker.TajoWorker; - -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class TajoContainerProxy extends ContainerProxy { - private final QueryContext queryContext; - private final TajoWorker.WorkerContext workerContext; - private final String planJson; - - public TajoContainerProxy(QueryMasterTask.QueryMasterTaskContext context, - Configuration conf, TajoContainer container, - QueryContext queryContext, ExecutionBlockId executionBlockId, String planJson) { - super(context, conf, executionBlockId, container); - this.queryContext = queryContext; - this.workerContext = context.getQueryMasterContext().getWorkerContext(); - this.planJson = planJson; - } - - @Override - public synchronized void launch(ContainerLaunchContext containerLaunchContext) { - //context.getResourceAllocator().addContainer(containerId, this); - - this.hostName = container.getNodeId().getHost(); - this.port = ((TajoWorkerContainer)container).getWorkerResource().getConnectionInfo().getPullServerPort(); - this.state = ContainerState.RUNNING; - - if (LOG.isDebugEnabled()) { - LOG.debug("Launch Container:" + executionBlockId + "," + containerId.getId() + "," + - container.getId() + "," + container.getNodeId() + ", pullServer=" + port); - } - - assignExecutionBlock(executionBlockId, container); - } - - /** - * It sends a kill RPC request to a corresponding worker. - * - * @param taskAttemptId The TaskAttemptId to be killed. - */ - public void killTaskAttempt(TaskAttemptId taskAttemptId) { - NettyClientBase tajoWorkerRpc = null; - try { - InetSocketAddress addr = new InetSocketAddress(container.getNodeId().getHost(), container.getNodeId().getPort()); - tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); - TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); - tajoWorkerRpcClient.killTaskAttempt(null, taskAttemptId.getProto(), NullCallback.get()); - } catch (Throwable e) { - /* Worker RPC failure */ - context.getEventHandler().handle(new TaskFatalErrorEvent(taskAttemptId, e.getMessage())); - } - } - - private void assignExecutionBlock(ExecutionBlockId executionBlockId, TajoContainer container) { - NettyClientBase tajoWorkerRpc; - try { - - InetSocketAddress addr = new InetSocketAddress(container.getNodeId().getHost(), container.getNodeId().getPort()); - tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); - TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); - - PlanProto.ShuffleType shuffleType = - context.getQuery().getStage(executionBlockId).getDataChannel().getShuffleType(); - -// TajoWorkerProtocol.RunExecutionBlockRequestProto request = -// TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder() -// .setExecutionBlockId(executionBlockId.getProto()) -// .setQueryMaster(context.getQueryMasterContext().getWorkerContext().getConnectionInfo().getProto()) -// .setNodeId(container.getNodeId().toString()) -// .setContainerId(container.getId().toString()) -// .setQueryOutputPath(context.getStagingDir().toString()) -// .setQueryContext(queryContext.getProto()) -// .setPlanJson(planJson) -// .setShuffleType(shuffleType) -// .build(); -// -// tajoWorkerRpcClient.startExecutionBlock(null, request, NullCallback.get()); - } catch (Throwable e) { - LOG.error(e.getMessage(), e); - } - } - - @Override - public synchronized void stopContainer() { - if (LOG.isDebugEnabled()) { - LOG.debug("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, Arrays.asList(containerId)); - //context.getResourceAllocator().removeContainer(containerId); - } catch (Throwable t) { - // ignore the cleanup failure - String message = "cleanup failed for container " - + this.containerId + " : " - + StringUtils.stringifyException(t); - LOG.warn(message); - } finally { - this.state = ContainerState.DONE; - } - } - } - - public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext context, - ExecutionBlockId executionBlockId, - List containerIds) throws Exception { - List containerIdProtos = - new ArrayList(); - - for(TajoContainerId eachContainerId: containerIds) { - containerIdProtos.add(TajoWorkerContainerId.getContainerIdProto(eachContainerId)); - } - - RpcClientManager manager = RpcClientManager.getInstance(); - NettyClientBase tmClient = null; - - ServiceTracker serviceTracker = context.getQueryMasterContext().getWorkerContext().getServiceTracker(); - tmClient = manager.getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); - - QueryCoordinatorProtocol.QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); -// masterClientService.releaseWorkerResource(null, -// QueryCoordinatorProtocol.WorkerResourceReleaseRequest.newBuilder() -// .setExecutionBlockId(executionBlockId.getProto()) -// .addAllContainerIds(containerIdProtos) -// .build(), -// NullCallback.get()); - - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index e9062cdfa5..955e03ee4a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.service.CompositeService; -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.util.Clock; @@ -40,7 +39,6 @@ import org.apache.tajo.engine.function.FunctionLoader; import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.master.rm.TajoResourceManager; -import org.apache.tajo.master.rm.WorkerResourceManager; import org.apache.tajo.metrics.CatalogMetricsGaugeSet; import org.apache.tajo.metrics.WorkerResourceMetricsGaugeSet; import org.apache.tajo.rpc.RpcChannelFactory; @@ -67,7 +65,6 @@ import java.lang.management.ManagementFactory; import java.lang.management.ThreadInfo; import java.lang.management.ThreadMXBean; -import java.lang.reflect.Constructor; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; @@ -157,59 +154,52 @@ public TajoMasterClientService getTajoMasterClientService() { } @Override - public void serviceInit(Configuration _conf) throws Exception { - if (!(_conf instanceof TajoConf)) { - throw new IllegalArgumentException("_conf should be a TajoConf type."); - } - this.systemConf = (TajoConf) _conf; + public void serviceInit(Configuration conf) throws Exception { + + this.systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook())); context = new MasterContext(systemConf); clock = new SystemClock(); - try { - RackResolver.init(systemConf); + RackResolver.init(systemConf); - RpcClientManager rpcManager = RpcClientManager.getInstance(); - rpcManager.setRetries(systemConf.getInt(RpcConstants.RPC_CLIENT_RETRY_MAX, RpcConstants.DEFAULT_RPC_RETRIES)); - rpcManager.setTimeoutSeconds( - systemConf.getInt(RpcConstants.RPC_CLIENT_TIMEOUT_SECS, RpcConstants.DEFAULT_RPC_TIMEOUT_SECONDS)); + RpcClientManager rpcManager = RpcClientManager.getInstance(); + rpcManager.setRetries(systemConf.getInt(RpcConstants.RPC_CLIENT_RETRY_MAX, RpcConstants.DEFAULT_RPC_RETRIES)); + rpcManager.setTimeoutSeconds( + systemConf.getInt(RpcConstants.RPC_CLIENT_TIMEOUT_SECS, RpcConstants.DEFAULT_RPC_TIMEOUT_SECONDS)); - initResourceManager(); + initResourceManager(); - this.dispatcher = new AsyncDispatcher(); - addIfService(dispatcher); + this.dispatcher = new AsyncDispatcher(); + addIfService(dispatcher); - // check the system directory and create if they are not created. - checkAndInitializeSystemDirectories(); - diagnoseTajoMaster(); - this.storeManager = TableSpaceManager.getFileStorageManager(systemConf); + // check the system directory and create if they are not created. + checkAndInitializeSystemDirectories(); + diagnoseTajoMaster(); + this.storeManager = TableSpaceManager.getFileStorageManager(systemConf); - catalogServer = new CatalogServer(loadFunctions()); - addIfService(catalogServer); - catalog = new LocalCatalogWrapper(catalogServer, systemConf); + catalogServer = new CatalogServer(loadFunctions()); + addIfService(catalogServer); + catalog = new LocalCatalogWrapper(catalogServer, systemConf); - sessionManager = new SessionManager(dispatcher); - addIfService(sessionManager); + sessionManager = new SessionManager(dispatcher); + addIfService(sessionManager); - globalEngine = new GlobalEngine(context); - addIfService(globalEngine); + globalEngine = new GlobalEngine(context); + addIfService(globalEngine); - queryManager = new QueryManager(context); - addIfService(queryManager); + queryManager = new QueryManager(context); + addIfService(queryManager); - tajoMasterClientService = new TajoMasterClientService(context); - addIfService(tajoMasterClientService); + tajoMasterClientService = new TajoMasterClientService(context); + addIfService(tajoMasterClientService); - tajoMasterService = new QueryCoordinatorService(context); - addIfService(tajoMasterService); - - restServer = new TajoRestService(context); - addIfService(restServer); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - throw e; - } + tajoMasterService = new QueryCoordinatorService(context); + addIfService(tajoMasterService); + + restServer = new TajoRestService(context); + addIfService(restServer); super.serviceInit(systemConf); LOG.info("Tajo Master is initialized."); @@ -382,39 +372,19 @@ private void checkBaseTBSpaceAndDatabase() throws IOException { } @Override - public void stop() { - if (haService != null) { - try { - haService.delete(); - } catch (Exception e) { - LOG.error(e, e); - } - } - - if (restServer != null) { - try { - restServer.stop(); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - } + public void serviceStop() throws Exception { + if (haService != null) haService.delete(); - if (webServer != null) { - try { - webServer.stop(); - } catch (Exception e) { - LOG.error(e, e); - } - } + if (restServer != null) restServer.stop(); + + if (webServer != null) webServer.stop(); IOUtils.cleanup(LOG, catalogServer); - if(systemMetrics != null) { - systemMetrics.stop(); - } + if (systemMetrics != null) systemMetrics.stop(); - if(pauseMonitor != null) pauseMonitor.stop(); - super.stop(); + if (pauseMonitor != null) pauseMonitor.stop(); + super.serviceStop(); LOG.info("Tajo Master main thread exiting"); } @@ -596,7 +566,7 @@ public static void main(String[] args) throws Exception { try { TajoMaster master = new TajoMaster(); - TajoConf conf = new TajoConf(new YarnConfiguration()); + TajoConf conf = new TajoConf(); master.init(conf); master.start(); } catch (Throwable t) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java index 0b9a3f85c7..9c294508bb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java @@ -48,13 +48,11 @@ import org.apache.tajo.master.exec.NonForwardQueryResultFileScanner; import org.apache.tajo.master.exec.NonForwardQueryResultScanner; import org.apache.tajo.master.rm.Worker; -import org.apache.tajo.master.rm.WorkerResource; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.logical.PartitionedTableScanNode; import org.apache.tajo.plan.logical.ScanNode; import org.apache.tajo.querymaster.QueryJobEvent; import org.apache.tajo.rpc.BlockingRpcServer; -import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListProto; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto; @@ -96,32 +94,27 @@ public TajoMasterClientService(MasterContext context) { } @Override - public void start() { + public void serviceStart() throws Exception { // start the rpc server String confClientServiceAddr = conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS); InetSocketAddress initIsa = NetUtils.createSocketAddr(confClientServiceAddr); int workerNum = conf.getIntVar(ConfVars.MASTER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM); - try { - server = new BlockingRpcServer(TajoMasterClientProtocol.class, clientHandler, initIsa, workerNum); - } catch (Exception e) { - LOG.error(e); - throw new RuntimeException(e); - } + server = new BlockingRpcServer(TajoMasterClientProtocol.class, clientHandler, initIsa, workerNum); server.start(); bindAddress = NetUtils.getConnectAddress(server.getListenAddress()); this.conf.setVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress)); + super.serviceStart(); LOG.info("Instantiated TajoMasterClientService at " + this.bindAddress); - super.start(); } @Override - public void stop() { + public void serviceStop() throws Exception { if (server != null) { server.shutdown(); } - super.stop(); + super.serviceStop(); } public InetSocketAddress getBindAddress() { @@ -591,7 +584,7 @@ public GetQueryInfoResponse getQueryInfo(RpcController controller, QueryIdReques QueryManager queryManager = context.getQueryJobManager(); QueryInProgress queryInProgress = queryManager.getQueryInProgress(queryId); - QueryInfo queryInfo = null; + QueryInfo queryInfo; if (queryInProgress == null) { queryInfo = context.getQueryJobManager().getFinishedQuery(queryId); } else { @@ -637,34 +630,23 @@ public GetClusterInfoResponse getClusterInfo(RpcController controller, context.getSessionManager().touch(request.getSessionId().getId()); GetClusterInfoResponse.Builder builder= GetClusterInfoResponse.newBuilder(); - Map workers = context.getResourceManager().getWorkers(); - - List wokerKeys = new ArrayList(workers.keySet()); - Collections.sort(wokerKeys); - - WorkerResourceInfo.Builder workerBuilder - = WorkerResourceInfo.newBuilder(); - //FIXME -// for(Worker worker: workers.values()) { -// WorkerResource workerResource = worker.getAvailableResource(); -// -// workerBuilder.setConnectionInfo(worker.getConnectionInfo().getProto()); -// workerBuilder.setDiskSlots(workerResource.getDiskSlots()); -// workerBuilder.setCpuCoreSlots(workerResource.getCpuCoreSlots()); -// workerBuilder.setMemoryMB(workerResource.getMemoryMB()); -// workerBuilder.setLastHeartbeat(worker.getLastHeartbeatTime()); -// workerBuilder.setUsedMemoryMB(workerResource.getUsedMemoryMB()); -// workerBuilder.setUsedCpuCoreSlots(workerResource.getUsedCpuCoreSlots()); -// workerBuilder.setUsedDiskSlots(workerResource.getUsedDiskSlots()); -// workerBuilder.setWorkerStatus(worker.getState().toString()); -// workerBuilder.setMaxHeap(workerResource.getMaxHeap()); -// workerBuilder.setFreeHeap(workerResource.getFreeHeap()); -// workerBuilder.setTotalHeap(workerResource.getTotalHeap()); -// workerBuilder.setNumRunningTasks(workerResource.getNumRunningTasks()); -// workerBuilder.setNumQueryMasterTasks(workerResource.getNumQueryMasterTasks()); -// -// builder.addWorkerList(workerBuilder.build()); -// } + List workers = new ArrayList(context.getResourceManager().getRMContext().getWorkers().values()); + Collections.sort(workers); + + WorkerResourceInfo.Builder workerBuilder = WorkerResourceInfo.newBuilder(); + + for(Worker worker: workers) { + workerBuilder.setConnectionInfo(worker.getConnectionInfo().getProto()); + workerBuilder.setAvailableResource(worker.getAvailableResource().getProto()); + workerBuilder.setTotalResource(worker.getTotalResourceCapability().getProto()); + + workerBuilder.setLastHeartbeat(worker.getLastHeartbeatTime()); + workerBuilder.setWorkerStatus(worker.getState().toString()); + workerBuilder.setNumRunningTasks(worker.getNumRunningTasks()); + workerBuilder.setNumQueryMasterTasks(worker.getNumRunningQueryMaster()); + + builder.addWorkerList(workerBuilder.build()); + } return builder.build(); } catch (Throwable t) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java deleted file mode 100644 index c1c6522e68..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java +++ /dev/null @@ -1,51 +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.hadoop.yarn.event.AbstractEvent; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.master.TaskRunnerGroupEvent.EventType; -import org.apache.tajo.master.container.TajoContainer; - -import java.util.Collection; - -public class TaskRunnerGroupEvent extends AbstractEvent { - public enum EventType { - CONTAINER_REMOTE_LAUNCH, - CONTAINER_REMOTE_CLEANUP - } - - protected final ExecutionBlockId executionBlockId; - protected final Collection containers; - public TaskRunnerGroupEvent(EventType eventType, - ExecutionBlockId executionBlockId, - Collection containers) { - super(eventType); - this.executionBlockId = executionBlockId; - this.containers = containers; - } - - public Collection getContainers() { - return containers; - } - - public ExecutionBlockId getExecutionBlockId() { - return executionBlockId; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java deleted file mode 100644 index c3a9a5981c..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java +++ /dev/null @@ -1,77 +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.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.ExecutionBlockId; - -public class ContainerAllocationEvent extends AbstractEvent { - - private final ExecutionBlockId executionBlockId; - private final Priority priority; - private final Resource resource; - private final boolean isLeafQuery; - private final int requiredNum; - private final float progress; - - public ContainerAllocationEvent(ContainerAllocatorEventType eventType, - ExecutionBlockId executionBlockId, - Priority priority, - Resource resource, - int requiredNum, - boolean isLeafQuery, float progress) { - super(eventType); - this.executionBlockId = executionBlockId; - this.priority = priority; - this.resource = resource; - this.requiredNum = requiredNum; - this.isLeafQuery = isLeafQuery; - this.progress = progress; - } - - public ExecutionBlockId getExecutionBlockId() { - return executionBlockId; - } - - public Priority getPriority() { - return priority; - } - - public int getRequiredNum() { - return requiredNum; - } - - public boolean isLeafQuery() { - return isLeafQuery; - } - - public Resource getCapability() { - return resource; - } - - public float getProgress() { - return progress; - } - - public Resource getResource() { - return resource; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java deleted file mode 100644 index 723ac1a7a3..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerEvent.java +++ /dev/null @@ -1,37 +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.event; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.tajo.master.event.ContainerEvent.EventType; - -public class ContainerEvent extends AbstractEvent { - public enum EventType { - CONTAINER_LAUNCHED, - CONTAINER_STOPPED - } - - private final ContainerId cId; - - public ContainerEvent(EventType eventType, ContainerId cId) { - super(eventType); - this.cId = cId; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java deleted file mode 100644 index c34b1743a6..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java +++ /dev/null @@ -1,45 +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.event; - -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.tajo.ExecutionBlockId; - -import java.util.Map; - -public class GrouppedContainerAllocatorEvent - extends ContainerAllocationEvent { - private final Map requestMap; - - public GrouppedContainerAllocatorEvent(ContainerAllocatorEventType eventType, - ExecutionBlockId executionBlockId, - Priority priority, - Resource resource, - Map requestMap, - boolean isLeafQuery, float progress) { - super(eventType, executionBlockId, priority, - resource, requestMap.size(), isLeafQuery, progress); - this.requestMap = requestMap; - } - - public Map getRequestMap() { - return this.requestMap; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java index 5cf98871a3..f3cc52aa7d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java @@ -27,20 +27,20 @@ */ public class LocalTaskEvent extends AbstractEvent { private final TaskAttemptId taskAttemptId; - private final TajoContainerId containerId; + private final int workerId; - public LocalTaskEvent(TaskAttemptId taskAttemptId, TajoContainerId containerId, + public LocalTaskEvent(TaskAttemptId taskAttemptId, int workerId, LocalTaskEventType eventType) { super(eventType); this.taskAttemptId = taskAttemptId; - this.containerId = containerId; + this.workerId = workerId; } public TaskAttemptId getTaskAttemptId() { return taskAttemptId; } - public TajoContainerId getContainerId() { - return containerId; + public int getWorkerId() { + return workerId; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java index 3a387fa9b7..9ce7f0945f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java @@ -21,6 +21,7 @@ import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.QueryId; import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.resource.NodeResource; import org.apache.tajo.session.Session; /** @@ -36,15 +37,17 @@ public enum EventType { private final QueryContext queryContext; private final String jsonExpr; private final String logicalPlanJson; + private final NodeResource allocation; public QueryStartEvent(QueryId queryId, Session session, QueryContext queryContext, String jsonExpr, - String logicalPlanJson) { + String logicalPlanJson, NodeResource allocation) { super(EventType.QUERY_START); this.queryId = queryId; this.session = session; this.queryContext = queryContext; this.jsonExpr = jsonExpr; this.logicalPlanJson = logicalPlanJson; + this.allocation = allocation; } public QueryId getQueryId() { @@ -67,6 +70,10 @@ public String getLogicalPlanJson() { return logicalPlanJson; } + public NodeResource getAllocation() { + return allocation; + } + @Override public String toString() { return getClass().getName() + "," + getType() + "," + queryId; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.java index 763d42655a..d9beaa80eb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageEventType.java @@ -26,9 +26,7 @@ public enum StageEventType { // Producer: Query SQ_INIT, SQ_START, - SQ_CONTAINER_ALLOCATED, SQ_KILL, - SQ_LAUNCH, // Producer: Task SQ_TASK_COMPLETED, diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java index 16ceccc2ce..c686078f54 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java @@ -18,14 +18,10 @@ package org.apache.tajo.master.event; -import com.google.protobuf.RpcCallback; import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.ipc.QueryCoordinatorProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType; -import org.apache.tajo.master.container.TajoContainerId; public class TaskRequestEvent extends AbstractEvent { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java index 6a82f294d8..7d90498386 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java @@ -42,7 +42,6 @@ import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.master.TajoMaster.MasterContext; import org.apache.tajo.master.rm.Worker; -import org.apache.tajo.master.rm.WorkerResource; import org.apache.tajo.plan.InvalidQueryException; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; @@ -51,6 +50,7 @@ import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.ScanNode; import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; import org.apache.tajo.session.InvalidSessionException; import org.apache.tajo.storage.RowStoreUtil; import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder; @@ -466,7 +466,9 @@ private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) { List columns = outSchema.getRootColumns(); Tuple aTuple = new VTuple(outSchema.size()); - NodeResource aResource = aWorker.getAvailableResource(); + + NodeResource total = aWorker.getTotalResourceCapability(); + NodeResource used = NodeResources.subtract(total, aWorker.getAvailableResource()); for (int fieldId = 0; fieldId < columns.size(); fieldId++) { Column column = columns.get(fieldId); @@ -489,15 +491,15 @@ private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) { aTuple.put(fieldId, DatumFactory.createText(aWorker.getState().toString())); } else if ("RUNNING".equalsIgnoreCase(aWorker.getState().toString())) { if ("total_cpu".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aResource.getVirtualCores())); + aTuple.put(fieldId, DatumFactory.createInt4(total.getVirtualCores())); } else if ("used_mem".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aResource.getMemory() * 1048576l)); + aTuple.put(fieldId, DatumFactory.createInt8(used.getMemory() * 1048576l)); } else if ("total_mem".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt8(aWorker.getTotalResourceCapability().getMemory() * 1048576l)); - } else if ("used_diskslots".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aResource.getDisks())); - } else if ("total_diskslots".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getTotalResourceCapability().getDisks())); + aTuple.put(fieldId, DatumFactory.createInt8(total.getMemory() * 1048576l)); + } else if ("used_disk".equalsIgnoreCase(column.getSimpleName())) { + aTuple.put(fieldId, DatumFactory.createInt4(used.getDisks())); + } else if ("total_disk".equalsIgnoreCase(column.getSimpleName())) { + aTuple.put(fieldId, DatumFactory.createInt4(total.getDisks())); } else if ("running_tasks".equalsIgnoreCase(column.getSimpleName())) { aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getNumRunningTasks())); } else if ("last_heartbeat_ts".equalsIgnoreCase(column.getSimpleName())) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java index 3c8f8b0626..41aea78c81 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java @@ -20,8 +20,6 @@ import com.google.common.collect.Maps; import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.tajo.QueryId; -import org.apache.tajo.ipc.ContainerProtocol; import java.util.Collections; import java.util.Set; @@ -41,15 +39,9 @@ public class TajoRMContext { /** map between workerIds and inactive workers */ private final ConcurrentMap inactiveWorkers = Maps.newConcurrentMap(); - /** map between queryIds and query master ContainerId */ - private final ConcurrentMap qmContainerMap = Maps - .newConcurrentMap(); - private final Set liveQueryMasterWorkerResources = Collections.newSetFromMap(new ConcurrentHashMap()); - private final Set stoppedQueryIds = - Collections.newSetFromMap(new ConcurrentHashMap()); public TajoRMContext(Dispatcher dispatcher) { this.rmDispatcher = dispatcher; @@ -73,19 +65,7 @@ public ConcurrentMap getInactiveWorkers() { return inactiveWorkers; } - /** - * - * @return The Map for query master containers - */ - public ConcurrentMap getQueryMasterContainer() { - return qmContainerMap; - } - public Set getQueryMasterWorker() { return liveQueryMasterWorkerResources; } - - public Set getStoppedQueryIds() { - return stoppedQueryIds; - } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java index b8c3042652..4fd6880fb6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java @@ -29,14 +29,16 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.master.TajoMaster; +import org.apache.tajo.master.scheduler.AbstractQueryScheduler; import org.apache.tajo.master.scheduler.QuerySchedulingInfo; import org.apache.tajo.master.scheduler.SimpleScheduler; -import org.apache.tajo.master.scheduler.TajoScheduler; import org.apache.tajo.master.scheduler.event.SchedulerEventType; import org.apache.tajo.util.TUtil; import java.io.IOException; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; /** * It manages all resources of tajo workers. @@ -57,7 +59,7 @@ public class TajoResourceManager extends CompositeService { private WorkerLivelinessMonitor workerLivelinessMonitor; private TajoConf systemConf; - private TajoScheduler scheduler; + private AbstractQueryScheduler scheduler; /** It receives status messages from workers and their resources. */ private TajoResourceTracker resourceTracker; @@ -67,6 +69,7 @@ public TajoResourceManager(TajoMaster.MasterContext masterContext) { this.masterContext = masterContext; } + @VisibleForTesting public TajoResourceManager(TajoConf systemConf) { super(TajoResourceManager.class.getSimpleName()); } @@ -154,7 +157,7 @@ TajoResourceTracker getResourceTracker() { return resourceTracker; } - public TajoScheduler getScheduler() { + public AbstractQueryScheduler getScheduler() { return scheduler; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index 7402913485..6b37a264e6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -32,8 +32,8 @@ import org.apache.tajo.resource.NodeResource; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.TUtil; -import java.io.IOError; import java.net.InetSocketAddress; import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; @@ -77,20 +77,14 @@ public TajoResourceTracker(TajoResourceManager manager, WorkerLivelinessMonitor @Override public void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - TajoConf systemConf = (TajoConf) conf; + + TajoConf systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); String confMasterServiceAddr = systemConf.getVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS); InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr); - try { - server = new AsyncRpcServer(TajoResourceTrackerProtocol.class, this, initIsa, 3); - } catch (Exception e) { - LOG.error(e); - throw new IOError(e); - } + int workerNum = systemConf.getIntVar(TajoConf.ConfVars.MASTER_RPC_SERVER_WORKER_THREAD_NUM); + server = new AsyncRpcServer(TajoResourceTrackerProtocol.class, this, initIsa, workerNum); server.start(); bindAddress = NetUtils.getConnectAddress(server.getListenAddress()); @@ -116,7 +110,8 @@ private static WorkerStatusEvent createStatusEvent(NodeHeartbeatRequestProto hea heartbeat.getWorkerId(), heartbeat.getRunningTasks(), heartbeat.getRunningQueryMasters(), - new NodeResource(heartbeat.getAvailableResource())); + new NodeResource(heartbeat.getAvailableResource()), + heartbeat.hasTotalResource() ? new NodeResource(heartbeat.getTotalResource()) : null); } @Override @@ -136,6 +131,8 @@ public void nodeHeartbeat( if (heartbeat.hasAvailableResource()) { // status update rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(heartbeat)); + + //refresh scheduler resource rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java index 5494c00e6f..3660689513 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java @@ -31,7 +31,6 @@ import java.util.EnumSet; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLongFieldUpdater; -import java.util.concurrent.locks.ReentrantReadWriteLock; /** * It contains resource and various information for a worker. @@ -237,7 +236,11 @@ private void updateStatus(WorkerStatusEvent statusEvent) { setLastHeartbeatTime(System.currentTimeMillis()); setNumRunningTasks(statusEvent.getRunningTaskNum()); setNumRunningQueryMaster(statusEvent.getRunningQMNum()); - NodeResources.update(availableResource, statusEvent.getResource()); + NodeResources.update(availableResource, statusEvent.getAvailableResource()); + + if(statusEvent.getTotalResource() != null) { + NodeResources.update(totalResourceCapability, statusEvent.getTotalResource()); + } } public static class DeactivateNodeTransition implements SingleArcTransition { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java deleted file mode 100644 index f05362b72b..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java +++ /dev/null @@ -1,110 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.master.rm; - -import org.apache.hadoop.service.Service; -import org.apache.tajo.QueryId; -import org.apache.tajo.ipc.ContainerProtocol; - -import java.io.IOException; -import java.util.Collection; -import java.util.Map; - -/** - * An interface of WorkerResourceManager which allows TajoMaster to request allocation for containers - * and release the allocated containers. - */ -public interface WorkerResourceManager extends Service { - - /** - * Request a resource container for a QueryMaster. - * - * @param queryInProgress QueryInProgress - * @return A allocated container resource - */ - //@Deprecated - //public QueryCoordinatorProtocol.WorkerAllocatedResource allocateQueryMaster(QueryInProgress queryInProgress); - - /** - * Request one or more resource containers. You can set the number of containers and resource capabilities, such as - * memory, CPU cores, and disk slots. This is an asynchronous call. You should use a callback to get allocated - * resource containers. Each container is identified {@link org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto}. - * - * @param request Request description - * @param rpcCallBack Callback function - */ -// public void allocateWorkerResources(QueryCoordinatorProtocol.NodeResourceRequestProto request, -// RpcCallback rpcCallBack); - - /** - * Release a container - * - * @param containerId ContainerIdProto to be released - */ - public void releaseWorkerResource(ContainerProtocol.TajoContainerIdProto containerId); - - public String getSeedQueryId() throws IOException; - - /** - * Check if a query master is stopped. - * - * @param queryId QueryId to be checked - * @return True if QueryMaster is stopped - */ - public boolean isQueryMasterStopped(QueryId queryId); - - /** - * Stop a query master - * - * @param queryId QueryId to be stopped - */ - public void releaseQueryMaster(QueryId queryId); - - /** - * - * @return a Map instance containing active workers - */ - public Map getWorkers(); - - /** - * - * @return a Map instance containing inactive workers - */ - public Map getInactiveWorkers(); - - public void stop(); - - /** - * - * @return The overall summary of cluster resources - */ - //public ClusterResourceSummary getClusterResourceSummary(); - - /** - * - * @return WorkerIds on which QueryMasters are running - */ - public Collection getQueryMasters(); - - /** - * - * @return RMContext - */ - public TajoRMContext getRMContext(); -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java index 862b4a1be7..11cf57e680 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java @@ -18,6 +18,7 @@ package org.apache.tajo.master.rm; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.resource.NodeResource; /** @@ -28,12 +29,15 @@ public class WorkerStatusEvent extends WorkerEvent { private final int runningTaskNum; private final int runningQMNum; private final NodeResource available; + private final NodeResource total; - public WorkerStatusEvent(int workerId, int runningTaskNum, int runningQMNum, NodeResource available) { + public WorkerStatusEvent(int workerId, int runningTaskNum, int runningQMNum, + NodeResource available, @Nullable NodeResource total) { super(workerId, WorkerEventType.STATE_UPDATE); this.runningTaskNum = runningTaskNum; this.runningQMNum = runningQMNum; this.available = available; + this.total = total; } public int getRunningTaskNum() { @@ -44,7 +48,11 @@ public int getRunningQMNum() { return runningQMNum; } - public NodeResource getResource() { + public NodeResource getAvailableResource() { return available; } + + public NodeResource getTotalResource() { + return total; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java new file mode 100644 index 0000000000..0fe614514e --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java @@ -0,0 +1,66 @@ +/** + * 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.scheduler; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.tajo.QueryId; +import org.apache.tajo.master.TajoMaster; +import org.apache.tajo.master.rm.TajoRMContext; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.resource.ResourceCalculator; + +public abstract class AbstractQueryScheduler extends AbstractService implements TajoResourceScheduler { + + protected final NodeResource clusterResource; + protected final NodeResource minResource; + protected final NodeResource maxResource; + + public AbstractQueryScheduler(String name) { + super(name); + this.minResource = NodeResources.createResource(0); + this.maxResource = NodeResources.createResource(0); + this.clusterResource = NodeResources.createResource(0); + } + + @Override + public NodeResource getClusterResource() { + return clusterResource; + } + + @Override + public NodeResource getMinimumResourceCapability() { + return minResource; + } + + @Override + public NodeResource getMaximumResourceCapability() { + return maxResource; + } + + public abstract ResourceCalculator getResourceCalculator(); + + public abstract void submitQuery(QuerySchedulingInfo schedulingInfo); + + public abstract void stopQuery(QueryId queryId); + +} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java index f88bc330d4..bb38ad44af 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java @@ -39,6 +39,10 @@ public QueryId getQueryId() { return queryId; } + public String getUser() { + return user; + } + public int getPriority() { return priority; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QueueInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QueueInfo.java new file mode 100644 index 0000000000..acf793cef6 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QueueInfo.java @@ -0,0 +1,101 @@ +/** + * 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.scheduler; + +import java.lang.String;import java.util.List; + +/** + * QueueInfo is a report of the runtime information of the queue. + *

+ * It includes information such as: + *

    + *
  • Queue name.
  • + *
  • Capacity of the queue.
  • + *
  • Maximum capacity of the queue.
  • + *
  • Current capacity of the queue.
  • + *
  • Child queues.
  • + *
  • Running applications.
  • + *
  • {@link QueueState} of the queue.
  • + *
+ * + */ + +public abstract class QueueInfo { + /** + * Get the name of the queue. + * @return name of the queue + */ + public abstract String getQueueName(); + + public abstract void setQueueName(String queueName); + + /** + * Get the configured capacity of the queue. + * @return configured capacity of the queue + */ + public abstract float getCapacity(); + + public abstract void setCapacity(float capacity); + + /** + * Get the maximum capacity of the queue. + * @return maximum capacity of the queue + */ + + public abstract float getMaximumCapacity(); + + public abstract void setMaximumCapacity(float maximumCapacity); + + /** + * Get the maximum query capacity of the queue. + * @return maximum query capacity of the queue + */ + + public abstract float getMaximumQueryCapacity(); + + public abstract void setMaximumQueryCapacity(float maximumQueryCapacity); + + /** + * Get the current capacity of the queue. + * @return current capacity of the queue + */ + + public abstract float getCurrentCapacity(); + + public abstract void setCurrentCapacity(float currentCapacity); + + /** + * Get the child queues of the queue. + * @return child queues of the queue + */ + + public abstract List getChildQueues(); + + public abstract void setChildQueues(List childQueues); + + + /** + * Get the QueueState of the queue. + * @return QueueState of the queue + */ + public abstract QueueState getQueueState(); + + public abstract void setQueueState(QueueState queueState); + +} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageContainerAllocationEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QueueState.java similarity index 50% rename from tajo-core/src/main/java/org/apache/tajo/master/event/StageContainerAllocationEvent.java rename to tajo-core/src/main/java/org/apache/tajo/master/scheduler/QueueState.java index 0d29e4467c..d7acbfe133 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/StageContainerAllocationEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QueueState.java @@ -16,23 +16,35 @@ * limitations under the License. */ -package org.apache.tajo.master.event; +package org.apache.tajo.master.scheduler; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.master.container.TajoContainer; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.records.QueueInfo; -import java.util.List; - -public class StageContainerAllocationEvent extends StageEvent { - private List allocatedContainer; - - public StageContainerAllocationEvent(final ExecutionBlockId id, - List allocatedContainer) { - super(id, StageEventType.SQ_CONTAINER_ALLOCATED); - this.allocatedContainer = allocatedContainer; - } - - public List getAllocatedContainer() { - return this.allocatedContainer; - } -} +/** + * State of a Queue. + *

+ * A queue is in one of: + *

    + *
  • {@link #RUNNING} - normal state.
  • + *
  • {@link #STOPPED} - not accepting new application submissions.
  • + *
+ * + * @see QueueInfo + * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest) + */ +@Public +@Stable +public enum QueueState { + /** + * Stopped - Not accepting submissions of new applications. + */ + STOPPED, + + /** + * Running - normal operation. + */ + RUNNING +} \ No newline at end of file diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java new file mode 100644 index 0000000000..4c111c71d2 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -0,0 +1,474 @@ +/** + * 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.scheduler; + +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.apache.hadoop.conf.Configuration; + +import org.apache.tajo.QueryId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.master.QueryInfo; +import org.apache.tajo.master.TajoMaster; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.master.rm.TajoRMContext; +import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.master.scheduler.event.ResourceReserveSchedulerEvent; +import org.apache.tajo.master.scheduler.event.SchedulerEvent; +import org.apache.tajo.resource.DefaultResourceCalculator; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.resource.ResourceCalculator; +import org.apache.tajo.util.TUtil; + +import java.util.*; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.PriorityBlockingQueue; + +import static org.apache.tajo.ipc.QueryCoordinatorProtocol.AllocationResourceProto; + +public class SimpleScheduler extends AbstractQueryScheduler { + + private static final Log LOG = LogFactory.getLog(SimpleScheduler.class); + + private static final String DEFAULT_QUEUE_NAME = "default"; + + + private volatile boolean isStopped = false; + private final TajoMaster.MasterContext masterContext; + private final TajoRMContext rmContext; + + private final BlockingQueue queryQueue; + private final Map pendingQueryMap = Maps.newHashMap(); + private final Map assignedQueryMasterMap = Maps.newHashMap(); + + private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator(); + private static Comparator COMPARATOR = new SchedulingAlgorithms.FifoComparator(); + + + private final Thread queryProcessor; + private QueueInfo queueInfo; + + private TajoConf tajoConf; + + public SimpleScheduler(TajoMaster.MasterContext context) { + super(SimpleScheduler.class.getName()); + this.masterContext = context; + this.rmContext = context.getResourceManager().getRMContext(); + this.queueInfo = new SimpleQueue(); + this.queryQueue = new PriorityBlockingQueue(11, COMPARATOR); + this.queryProcessor = new Thread(new QueryProcessor()); + } + + private void initScheduler(TajoConf conf) { + validateConf(conf); + int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); + this.minResource.setMemory(minQMMem).setVirtualCores(1); + this.queueInfo.setCapacity(1.0f); + this.queueInfo.setMaximumCapacity(queueInfo.getCapacity()); + this.queueInfo.setMaximumQueryCapacity(0.3f); // maximum parall + this.queueInfo.setQueueState(QueueState.RUNNING); + this.queueInfo.setChildQueues(new ArrayList()); + updateResource(); + this.queryProcessor.setName("Query Processor"); + } + + private void updateResource() { + NodeResource resource = NodeResources.createResource(0); + NodeResource totalResource = NodeResources.createResource(0); + for (Worker worker : getRMContext().getWorkers().values()) { + NodeResources.addTo(resource, worker.getAvailableResource()); + NodeResources.addTo(totalResource, worker.getTotalResourceCapability()); + + } + + NodeResources.update(maxResource, totalResource); + NodeResources.update(clusterResource, resource); + + if (getResourceCalculator().isInvalidDivisor(clusterResource)) { + this.queueInfo.setCurrentCapacity(0.0f); + } else { + this.queueInfo.setCurrentCapacity(getResourceCalculator().ratio(clusterResource, maxResource)); + } + + LOG.info("Scheduler resources \r current: " + getClusterResource() + + "\r maximum: " + getMaximumResourceCapability() + "\r queue: " + queueInfo); + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + this.tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); + initScheduler(tajoConf); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + this.queryProcessor.start(); + super.serviceStart(); + } + + @Override + public ResourceCalculator getResourceCalculator() { + return resourceCalculator; + } + + private void validateConf(TajoConf conf) { + // validate scheduler memory allocation setting + int minMem = conf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); + int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); + int maxMem = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); + + if (minMem <= 0 || minQMMem <= 0 || minMem + minQMMem > maxMem) { + throw new RuntimeException("Invalid resource scheduler memory" + + " allocation configuration" + + ", " + TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.varname + + "=" + minMem + + ", " + TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY.varname + + "=" + minQMMem + + ", " + TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname + + "=" + maxMem + ", min and max should be greater than 0" + + ", max should be no smaller than min."); + } + } + + private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceRequest(QueryInfo queryInfo) { + int qmMemory = tajoConf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); + + QueryCoordinatorProtocol.NodeResourceRequestProto.Builder builder = + QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); + + builder.setQueryId(queryInfo.getQueryId().getProto()) + .setCapacity(NodeResources.createResource(qmMemory).getProto()) + .setType(QueryCoordinatorProtocol.ResourceType.QUERYMASTER) + .setPriority(1) + .setNumContainers(1) + .setRunningTasks(1) + .setUserId(queryInfo.getQueryContext().getUser()); + //TODO .setQueue(queryInfo.getQueue()); + return builder.build(); + } + + + @Override + public int getNumClusterNodes() { + return rmContext.getWorkers().size(); + } + + @Override + public List + reserve(QueryId queryId, QueryCoordinatorProtocol.NodeResourceRequestProto request) { + if (LOG.isDebugEnabled()) { + LOG.debug("Request:" + request.toString() + "Cluster resource: " + getClusterResource()); + } + + List reservedResources = Lists.newArrayList(); + NodeResource capacity = new NodeResource(request.getCapacity()); + if (!NodeResources.fitsIn(capacity, getClusterResource())) { + return reservedResources; + } + + AllocationResourceProto.Builder resourceBuilder = AllocationResourceProto.newBuilder(); + LinkedList workers = new LinkedList(); + + if (request.getCandidateNodesCount() > 0) { + workers.addAll(request.getCandidateNodesList()); + } + + int allocatedResources = 0; + int requiredContainers = request.getNumContainers(); + + // reserve resource in candidate workers + while (workers.size() > 0) { + Iterator iter = workers.iterator(); + while (iter.hasNext()) { + + int workerId = iter.next(); + Worker worker = getRMContext().getWorkers().get(workerId); + if (worker == null) { + iter.remove(); + LOG.warn("Can't found the worker :" + workerId); + continue; + } else { + if (NodeResources.fitsIn(capacity, worker.getAvailableResource())) { + NodeResources.subtractFrom(getClusterResource(), capacity); + NodeResources.subtractFrom(worker.getAvailableResource(), capacity); + allocatedResources++; + resourceBuilder.setResource(capacity.getProto()); + resourceBuilder.setWorkerId(workerId); + reservedResources.add(resourceBuilder.build()); + } else { + // remove unavailable worker; + iter.remove(); + } + } + + if (allocatedResources >= requiredContainers) { + return reservedResources; + } + } + } + + // reserve resource in random workers + if(allocatedResources == 0 || allocatedResources < requiredContainers) { + LinkedList randomWorkers = new LinkedList(getRMContext().getWorkers().keySet()); + randomWorkers.removeAll(workers); + Collections.shuffle(randomWorkers); + + for (int workerId : randomWorkers) { + + Worker worker = getRMContext().getWorkers().get(workerId); + if (worker == null) { + LOG.warn("Can't found the worker :" + workerId); + continue; + } else { + if (NodeResources.fitsIn(capacity, worker.getAvailableResource())) { + NodeResources.subtractFrom(getClusterResource(), capacity); + NodeResources.subtractFrom(worker.getAvailableResource(), capacity); + allocatedResources++; + resourceBuilder.setResource(capacity.getProto()); + resourceBuilder.setWorkerId(workerId); + reservedResources.add(resourceBuilder.build()); + } + } + + if (allocatedResources >= requiredContainers) { + break; + } + } + } + + return reservedResources; + } + + + @Override + public void handle(SchedulerEvent event) { + switch (event.getType()) { + case RESOURCE_RESERVE: + reserveResource(TUtil.checkTypeAndGet(event, ResourceReserveSchedulerEvent.class)); + break; + case RESOURCE_UPDATE: + updateResource(); + break; + } + } + + /** + * This is an asynchronous call. You should use a callback to get reserved resource containers. + */ + protected void reserveResource(ResourceReserveSchedulerEvent schedulerEvent) { + List resources = + reserve(new QueryId(schedulerEvent.getRequest().getQueryId()), schedulerEvent.getRequest()); + + QueryCoordinatorProtocol.NodeResourceResponseProto.Builder response = + QueryCoordinatorProtocol.NodeResourceResponseProto.newBuilder(); + response.setQueryId(schedulerEvent.getRequest().getQueryId()); + schedulerEvent.getCallBack().run(response.addAllResource(resources).build()); + } + + /** + * Submit a query to scheduler + */ + public void submitQuery(QuerySchedulingInfo schedulingInfo) { + queryQueue.add(schedulingInfo); + pendingQueryMap.put(schedulingInfo.getQueryId(), schedulingInfo); + } + + public void stopQuery(QueryId queryId) { + if(pendingQueryMap.containsKey(queryId)){ + queryQueue.remove(pendingQueryMap.remove(queryId)); + } + assignedQueryMasterMap.remove(queryId); + } + + public BlockingQueue getQueryQueue() { + return queryQueue; + } + + private Worker getWorker(int workerId) { + return rmContext.getWorkers().get(workerId); + } + + protected TajoRMContext getRMContext() { + return rmContext; + } + + public WorkerConnectionInfo getQueryMaster(QueryId queryId) { + if (assignedQueryMasterMap.containsKey(queryId)) { + return rmContext.getWorkers().get(assignedQueryMasterMap.get(queryId)).getConnectionInfo(); + } + return null; + } + + private final class QueryProcessor implements Runnable { + @Override + public void run() { + + QuerySchedulingInfo query; + + while (!isStopped && !Thread.currentThread().isInterrupted()) { + try { + query = queryQueue.take(); + } catch (InterruptedException e) { + e.printStackTrace(); + break; + } + + //QueueInfo queueInfo = getQueueInfo(query.getQueue(), true, true); + + int maxAvailable = getResourceCalculator().computeAvailableContainers( + getMaximumResourceCapability(), getMinimumResourceCapability()); + + // limit maximum running queries +// if ((assignedQueryMasterMap.size() / maxAvailable) > queueInfo.getCurrentCapacity()) { + if (assignedQueryMasterMap.size() * 2 > maxAvailable) { + queryQueue.add(query); + synchronized (this) { + try { + this.wait(100); + } catch (InterruptedException e) { + if(!isStopped) { + LOG.fatal(e.getMessage(), e); + return; + } + } + } + } else { + QueryInfo queryInfo = + masterContext.getQueryJobManager().getQueryInProgress(query.getQueryId()).getQueryInfo(); + List allocation = + reserve(query.getQueryId(), createQMResourceRequest(queryInfo)); + + if(allocation.size() == 0) { + queryQueue.add(query); + synchronized (this) { + try { + this.wait(100); + } catch (InterruptedException e) { + LOG.fatal(e); + } + } + LOG.info("No Available Resources for QueryMaster :" + queryInfo.getQueryId() + "," + queryInfo); + } else { + try { + boolean started = masterContext.getQueryJobManager().startQueryJob(query.getQueryId(), allocation.get(0)); + if(!started) { + queryQueue.put(query); + } else { + assignedQueryMasterMap.put(query.getQueryId(), allocation.get(0).getWorkerId()); + } + } catch (Throwable t) { + LOG.fatal("Exception during query startup:", t); + masterContext.getQueryJobManager().stopQuery(query.getQueryId()); + } + } + } + LOG.info("Running Queries: " + assignedQueryMasterMap.size()); + } + } + } + + static class SimpleQueue extends QueueInfo { + private List childQueues; + private float capacity; + private float currentCapacity; + private float maximumCapacity; + private float maximumQueryCapacity; + private QueueState state; + + @Override + public String getQueueName() { + return DEFAULT_QUEUE_NAME; + } + + @Override + public void setQueueName(String queueName) { + } + + @Override + public float getCapacity() { + return capacity; + } + + @Override + public void setCapacity(float capacity) { + this.capacity = capacity; + } + + @Override + public float getMaximumCapacity() { + return maximumCapacity; + } + + @Override + public void setMaximumCapacity(float maximumCapacity) { + this.maximumCapacity = maximumCapacity; + } + + @Override + public float getMaximumQueryCapacity() { + return maximumQueryCapacity; + } + + @Override + public void setMaximumQueryCapacity(float maximumQueryCapacity) { + this.maximumQueryCapacity = maximumQueryCapacity; + } + + @Override + public float getCurrentCapacity() { + return currentCapacity; + } + + @Override + public void setCurrentCapacity(float currentCapacity) { + this.currentCapacity = currentCapacity; + } + + @Override + public List getChildQueues() { + return childQueues; + } + + @Override + public void setChildQueues(List childQueues) { + this.childQueues = childQueues; + } + + @Override + public QueueState getQueueState() { + return state; + } + + @Override + public void setQueueState(QueueState queueState) { + this.state = queueState; + } + + @Override + public String toString() { + return String.format("Queue name: %s, state: %s, maximum: %f, current: %f", + getQueueName(), getQueueState(), getMaximumCapacity(), getCurrentCapacity()); + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java new file mode 100644 index 0000000000..170a6e8ca9 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java @@ -0,0 +1,68 @@ +/** + * 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.scheduler; + +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.QueryId; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.master.scheduler.event.SchedulerEvent; +import org.apache.tajo.resource.NodeResource; + +import java.util.List; + +/** + * This interface is used by scheduler for allocating of resources. + */ +public interface TajoResourceScheduler extends EventHandler { + + /** + * Get the whole resource capacity of the cluster. + * @return the whole resource capacity of the cluster. + */ + + NodeResource getClusterResource(); + + /** + * Get minimum allocatable {@link NodeResource}. + * @return minimum allocatable resource + */ + NodeResource getMinimumResourceCapability(); + + /** + * Get maximum allocatable {@link NodeResource}. + * @return maximum allocatable resource + */ + NodeResource getMaximumResourceCapability(); + + /** + * Get the number of nodes available in the cluster. + * @return the number of available nodes. + */ + int getNumClusterNodes(); + + /** + * Get reservation resource. The cluster resource is updated by TajoResourceTracker + * Request one or more resource containers. You can set the number of containers and resource capabilities, + * such as memory, CPU cores, and disk slots. + * @return the number of reserved resources. + */ + List + reserve(QueryId queryId, QueryCoordinatorProtocol.NodeResourceRequestProto ask); + +} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java new file mode 100644 index 0000000000..80b3b3913d --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java @@ -0,0 +1,45 @@ +/** + * 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.scheduler.event; + +import com.google.protobuf.RpcCallback; +import static org.apache.tajo.ipc.QueryCoordinatorProtocol.NodeResourceRequestProto; +import static org.apache.tajo.ipc.QueryCoordinatorProtocol.NodeResourceResponseProto; + +public class ResourceReserveSchedulerEvent extends SchedulerEvent { + + private NodeResourceRequestProto request; + + private RpcCallback callBack; + + public ResourceReserveSchedulerEvent(NodeResourceRequestProto request, + RpcCallback callback) { + super(SchedulerEventType.RESOURCE_RESERVE); + this.request = request; + this.callBack = callback; + } + + public NodeResourceRequestProto getRequest() { + return request; + } + + public RpcCallback getCallBack() { + return callBack; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocatorEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEvent.java similarity index 76% rename from tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocatorEventType.java rename to tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEvent.java index 183aeb5aed..856c5725e4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/ContainerAllocatorEventType.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEvent.java @@ -16,11 +16,12 @@ * limitations under the License. */ -package org.apache.tajo.master.event; +package org.apache.tajo.master.scheduler.event; -public enum ContainerAllocatorEventType { - // producer: TaskAttempt, consumer: ContainerAllocator - CONTAINER_REQ, - CONTAINER_DEALLOCATE, - CONTAINER_FAILED +import org.apache.hadoop.yarn.event.AbstractEvent; + +public class SchedulerEvent extends AbstractEvent { + public SchedulerEvent(SchedulerEventType type) { + super(type); + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerLauncher.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEventType.java similarity index 81% rename from tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerLauncher.java rename to tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEventType.java index 9086e6588d..725cbcd12a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerLauncher.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEventType.java @@ -16,10 +16,16 @@ * limitations under the License. */ -package org.apache.tajo.master; +package org.apache.tajo.master.scheduler.event; -import org.apache.hadoop.yarn.event.EventHandler; +public enum SchedulerEventType { -public interface TaskRunnerLauncher extends EventHandler { + // Source: QM + RESOURCE_RESERVE, + RESOURCE_UPDATE, + + // Source: MASTER + QM_SUBMIT, + QM_STOP } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index eb5374be76..aab18bf8fb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -32,20 +32,16 @@ import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequest; import org.apache.tajo.engine.query.TaskRequestImpl; import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.ContainerProxy; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.container.TajoContainerId; -import org.apache.tajo.master.container.TajoContainerIdPBImpl; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; import org.apache.tajo.master.event.TaskSchedulerEvent.EventType; import org.apache.tajo.master.rm.TajoWorkerContainerId; -import org.apache.tajo.master.rm.Worker; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.serder.LogicalNodeSerializer; import org.apache.tajo.plan.serder.PlanProto; @@ -62,7 +58,6 @@ import org.apache.tajo.util.NetUtils; import org.apache.tajo.worker.FetchImpl; -import java.net.ConnectException; import java.net.InetSocketAddress; import java.util.*; import java.util.Map.Entry; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index 7e8bc43648..c1ebd802b8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -18,7 +18,6 @@ package org.apache.tajo.querymaster; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -29,20 +28,22 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.SystemClock; -import org.apache.tajo.*; +import org.apache.tajo.QueryId; +import org.apache.tajo.TajoProtos; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.QueryCoordinatorProtocol; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.ipc.QueryCoordinatorProtocol.QueryCoordinatorProtocolService; +import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeat; +import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeatResponse; +import org.apache.tajo.ipc.QueryCoordinatorProtocol.WorkerConnectionsProto; import org.apache.tajo.master.event.QueryStartEvent; import org.apache.tajo.master.event.QueryStopEvent; import org.apache.tajo.rpc.*; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.service.ServiceTracker; -import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.TUtil; import org.apache.tajo.util.history.QueryHistory; import org.apache.tajo.worker.TajoWorker; @@ -53,7 +54,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; public class QueryMaster extends CompositeService implements EventHandler { private static final Log LOG = LogFactory.getLog(QueryMaster.class.getName()); @@ -74,7 +74,7 @@ public class QueryMaster extends CompositeService implements EventHandler { private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread; - private AtomicBoolean queryMasterStop = new AtomicBoolean(false); + private volatile boolean isStopped; private QueryMasterContext queryMasterContext; @@ -95,39 +95,30 @@ public QueryMaster(TajoWorker.WorkerContext workerContext) { this.workerContext = workerContext; } - public void init(Configuration conf) { - LOG.info("QueryMaster init"); - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("conf should be a TajoConf type"); - } - try { - this.systemConf = (TajoConf)conf; - this.manager = RpcClientManager.getInstance(); + @Override + public void serviceInit(Configuration conf) throws Exception { - querySessionTimeout = systemConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT); - queryMasterContext = new QueryMasterContext(systemConf); + this.systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); + this.manager = RpcClientManager.getInstance(); - clock = new SystemClock(); + querySessionTimeout = systemConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT); + queryMasterContext = new QueryMasterContext(systemConf); - this.dispatcher = new AsyncDispatcher(); - addIfService(dispatcher); + clock = new SystemClock(); - globalPlanner = new GlobalPlanner(systemConf, workerContext); + this.dispatcher = new AsyncDispatcher(); + addIfService(dispatcher); - dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler()); - dispatcher.register(QueryStopEvent.EventType.class, new QueryStopEventHandler()); + globalPlanner = new GlobalPlanner(systemConf, workerContext); - } catch (Throwable t) { - LOG.error(t.getMessage(), t); - throw new RuntimeException(t); - } - super.init(conf); + dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler()); + dispatcher.register(QueryStopEvent.EventType.class, new QueryStopEventHandler()); + super.serviceInit(conf); + LOG.info("QueryMaster inited"); } @Override - public void start() { - LOG.info("QueryMaster start"); - + public void serviceStart() throws Exception { queryHeartbeatThread = new QueryHeartbeatThread(); queryHeartbeatThread.start(); @@ -137,15 +128,14 @@ public void start() { finishedQueryMasterTaskCleanThread = new FinishedQueryMasterTaskCleanThread(); finishedQueryMasterTaskCleanThread.start(); - eventExecutor = Executors.newSingleThreadExecutor(); - super.start(); + eventExecutor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); + super.serviceStart(); + LOG.info("QueryMaster started"); } @Override - public void stop() { - if(queryMasterStop.getAndSet(true)){ - return; - } + public void serviceStop() throws Exception { + isStopped = true; if(queryHeartbeatThread != null) { queryHeartbeatThread.interrupt(); @@ -163,59 +153,10 @@ public void stop() { eventExecutor.shutdown(); } - super.stop(); - + super.serviceStop(); LOG.info("QueryMaster stopped"); } - //FIXME remove this - protected void cleanupExecutionBlock(List executionBlockIds) { - StringBuilder cleanupMessage = new StringBuilder(); - String prefix = ""; - for (TajoIdProtos.ExecutionBlockIdProto eachEbId: executionBlockIds) { - cleanupMessage.append(prefix).append(new ExecutionBlockId(eachEbId).toString()); - prefix = ","; - } - LOG.info("cleanup executionBlocks: " + cleanupMessage); - NettyClientBase rpc = null; - List workers = getAllWorker(); - TajoWorkerProtocol.ExecutionBlockListProto.Builder builder = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); - builder.addAllExecutionBlockId(Lists.newArrayList(executionBlockIds)); - TajoWorkerProtocol.ExecutionBlockListProto executionBlockListProto = builder.build(); - - for (TajoProtos.WorkerConnectionInfoProto worker : workers) { - try { - rpc = manager.getClient(NetUtils.createSocketAddr(worker.getHost(), worker.getPeerRpcPort()), - TajoWorkerProtocol.class, true); - TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); - - tajoWorkerProtocolService.cleanupExecutionBlocks(null, executionBlockListProto, NullCallback.get()); - } catch (Exception e) { - LOG.warn("Ignoring exception. " + e.getMessage(), e); - continue; - } - } - } - - //FIXME get workers to QueryMasterTask - private void cleanup(QueryId queryId) { - LOG.info("cleanup query resources : " + queryId); - NettyClientBase rpc = null; - List workers = getAllWorker(); - - for (TajoProtos.WorkerConnectionInfoProto worker : workers) { - try { - rpc = manager.getClient(NetUtils.createSocketAddr(worker.getHost(), worker.getPeerRpcPort()), - TajoWorkerProtocol.class, true); - TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); - - tajoWorkerProtocolService.cleanup(null, queryId.getProto(), NullCallback.get()); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - } - } - public List getAllWorker() { NettyClientBase rpc = null; @@ -331,7 +272,7 @@ public void stopQuery(QueryId queryId) { TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(queryMasterTask); CallFuture future = new CallFuture(); - NettyClientBase tmClient = null; + NettyClientBase tmClient; try { tmClient = manager.getClient(workerContext.getServiceTracker().getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); @@ -348,9 +289,6 @@ public void stopQuery(QueryId queryId) { try { queryMasterTask.stop(); - if (!queryContext.getBool(SessionVars.DEBUG_ENABLED)) { - cleanup(queryId); - } } catch (Exception e) { LOG.error(e.getMessage(), e); } @@ -391,7 +329,7 @@ private class QueryStartEventHandler implements EventHandler { public void handle(QueryStartEvent event) { LOG.info("Start QueryStartEventHandler:" + event.getQueryId()); QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext, - event.getQueryId(), event.getSession(), event.getQueryContext(), event.getJsonExpr()); + event.getQueryId(), event.getSession(), event.getQueryContext(), event.getJsonExpr(), event.getAllocation()); synchronized(queryMasterTasks) { queryMasterTasks.put(event.getQueryId(), queryMasterTask); @@ -406,7 +344,6 @@ public void handle(QueryStartEvent event) { if (queryMasterTask.isInitError()) { queryMasterContext.stopQuery(queryMasterTask.getQueryId()); - return; } } } @@ -426,31 +363,29 @@ public QueryHeartbeatThread() { @Override public void run() { LOG.info("Start QueryMaster heartbeat thread"); - while(!queryMasterStop.get()) { + while(!isStopped) { List tempTasks = new ArrayList(); - synchronized(queryMasterTasks) { - tempTasks.addAll(queryMasterTasks.values()); - } - synchronized(queryMasterTasks) { - for(QueryMasterTask eachTask: tempTasks) { - NettyClientBase tmClient; - try { + tempTasks.addAll(queryMasterTasks.values()); - ServiceTracker serviceTracker = queryMasterContext.getWorkerContext().getServiceTracker(); - tmClient = manager.getClient(serviceTracker.getUmbilicalAddress(), - QueryCoordinatorProtocol.class, true); - QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); + for(QueryMasterTask eachTask: tempTasks) { + NettyClientBase tmClient; + try { - TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(eachTask); - masterClientService.heartbeat(null, queryHeartbeat, NullCallback.get()); - } catch (Throwable t) { - t.printStackTrace(); - } + ServiceTracker serviceTracker = queryMasterContext.getWorkerContext().getServiceTracker(); + tmClient = manager.getClient(serviceTracker.getUmbilicalAddress(), + QueryCoordinatorProtocol.class, true); + QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); + + TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(eachTask); + masterClientService.heartbeat(null, queryHeartbeat, NullCallback.get()); + } catch (Throwable t) { + t.printStackTrace(); } } - synchronized(queryMasterStop) { + + synchronized(this) { try { - queryMasterStop.wait(2000); + this.wait(2000); } catch (InterruptedException e) { break; } @@ -463,16 +398,16 @@ public void run() { class ClientSessionTimeoutCheckThread extends Thread { public void run() { LOG.info("ClientSessionTimeoutCheckThread started"); - while(!queryMasterStop.get()) { + while(!isStopped) { try { - Thread.sleep(1000); + synchronized (this) { + this.wait(1000); + } } catch (InterruptedException e) { break; } List tempTasks = new ArrayList(); - synchronized(queryMasterTasks) { - tempTasks.addAll(queryMasterTasks.values()); - } + tempTasks.addAll(queryMasterTasks.values()); for(QueryMasterTask eachTask: tempTasks) { if(!eachTask.isStopped()) { @@ -496,9 +431,11 @@ class FinishedQueryMasterTaskCleanThread extends Thread { public void run() { int expireIntervalTime = systemConf.getIntVar(TajoConf.ConfVars.QUERYMASTER_HISTORY_EXPIRE_PERIOD); LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime); - while(!queryMasterStop.get()) { + while(!isStopped) { try { - Thread.sleep(60 * 1000); // minimum interval minutes + synchronized (this) { + this.wait(60 * 1000); // minimum interval minutes + } } catch (InterruptedException e) { break; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index 1b15eea5a3..8e0b580616 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -18,7 +18,6 @@ package org.apache.tajo.querymaster; -import com.google.common.base.Preconditions; import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; import org.apache.commons.logging.Log; @@ -31,18 +30,18 @@ import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.container.TajoContainerId; import org.apache.tajo.master.event.*; -import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.session.Session; +import org.apache.tajo.resource.NodeResource; import org.apache.tajo.rpc.AsyncRpcServer; +import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; +import org.apache.tajo.session.Session; import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TajoWorker; import org.apache.tajo.worker.event.QMResourceAllocateEvent; import java.net.InetSocketAddress; -import java.util.concurrent.ExecutionException; public class QueryMasterManagerService extends CompositeService implements QueryMasterProtocol.QueryMasterProtocolService.Interface { @@ -68,78 +67,47 @@ public QueryMaster getQueryMaster() { } @Override - public void init(Configuration conf) { - Preconditions.checkArgument(conf instanceof TajoConf); - TajoConf tajoConf = (TajoConf) 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); - } - - int workerNum = tajoConf.getIntVar(TajoConf.ConfVars.QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM); - this.rpcServer = new AsyncRpcServer(QueryMasterProtocol.class, this, initIsa, workerNum); - this.rpcServer.start(); + public void serviceInit(Configuration conf) throws Exception { + + TajoConf tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); + // Setup RPC server + InetSocketAddress initIsa = + new InetSocketAddress("0.0.0.0", port); + if (initIsa.getAddress() == null) { + throw new IllegalArgumentException("Failed resolve of " + initIsa); + } - this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress()); - this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort(); + int workerNum = tajoConf.getIntVar(TajoConf.ConfVars.QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM); + this.rpcServer = new AsyncRpcServer(QueryMasterProtocol.class, this, initIsa, workerNum); + this.rpcServer.start(); - this.port = bindAddr.getPort(); + this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress()); + this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort(); - queryMaster = new QueryMaster(workerContext); - addService(queryMaster); + this.port = bindAddr.getPort(); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } + queryMaster = new QueryMaster(workerContext); + addService(queryMaster); // Get the master address LOG.info("QueryMasterManagerService is bind to " + addr); - ((TajoConf)conf).setVar(TajoConf.ConfVars.WORKER_QM_RPC_ADDRESS, addr); - - super.init(conf); - } + tajoConf.setVar(TajoConf.ConfVars.WORKER_QM_RPC_ADDRESS, addr); - @Override - public void start() { - super.start(); + super.serviceInit(conf); } @Override - public void stop() { + public void serviceStop() throws Exception { if(rpcServer != null) { rpcServer.shutdown(); } LOG.info("QueryMasterManagerService stopped"); - super.stop(); + super.serviceStop(); } public InetSocketAddress getBindAddr() { return bindAddr; } - @Override - public void getTask(RpcController controller, TajoWorkerProtocol.GetTaskRequestProto request, - RpcCallback done) { - try { - ExecutionBlockId ebId = new ExecutionBlockId(request.getExecutionBlockId()); - QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(ebId.getQueryId()); - - if(queryMasterTask == null || queryMasterTask.isStopped()) { - done.run(DefaultTaskScheduler.stopTaskRunnerReq); - } else { -// TajoContainerId cid = -// queryMasterTask.getQueryTaskContext().getResourceAllocator().makeContainerId(request.getContainerId()); -// LOG.debug("getTask:" + cid + ", ebId:" + ebId); -// queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(request.getWorkerId(), cid, ebId, done)); - } - } catch (Exception e) { - LOG.error(e.getMessage(), e); - controller.setFailed(e.getMessage()); - } - } - @Override public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatusProto request, RpcCallback done) { @@ -235,12 +203,12 @@ public void executeQuery(RpcController controller, new Session(request.getSession()), new QueryContext(workerContext.getQueryMaster().getContext().getConf(), request.getQueryContext()), request.getExprInJson().getValue(), - request.getLogicalPlanJson().getValue())); + request.getLogicalPlanJson().getValue(), new NodeResource(request.getAllocation().getResource()))); done.run(TajoWorker.NULL_PROTO); } @Override - public void startQueryMaster(RpcController controller, + public void allocateQueryMaster(RpcController controller, QueryCoordinatorProtocol.AllocationResourceProto request, RpcCallback done) { CallFuture callFuture = new CallFuture(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index b239de942d..c592c133be 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -19,6 +19,7 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -40,12 +41,9 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.exception.UnimplementedException; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.TajoContainerProxy; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; -import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.plan.LogicalOptimizer; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.LogicalPlanner; @@ -57,23 +55,24 @@ import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.verifier.VerifyException; import org.apache.tajo.resource.NodeResource; -import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.AsyncRpcClient; +import org.apache.tajo.rpc.NettyClientBase; +import org.apache.tajo.rpc.NullCallback; +import org.apache.tajo.rpc.RpcClientManager; import org.apache.tajo.session.Session; -import org.apache.tajo.storage.Tablespace; import org.apache.tajo.storage.StorageProperty; import org.apache.tajo.storage.StorageUtil; import org.apache.tajo.storage.TableSpaceManager; +import org.apache.tajo.storage.Tablespace; +import org.apache.tajo.util.TUtil; import org.apache.tajo.util.metrics.TajoMetrics; import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter; -import org.apache.tajo.worker.AbstractResourceAllocator; -import org.apache.tajo.worker.TajoResourceAllocator; import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.*; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import static org.apache.tajo.TajoProtos.QueryState; @@ -111,14 +110,14 @@ public class QueryMasterTask extends CompositeService { private AtomicLong lastClientHeartbeat = new AtomicLong(-1); -// private AbstractResourceAllocator resourceAllocator; - - private AtomicBoolean stopped = new AtomicBoolean(false); + private volatile boolean isStopped; private TajoMetrics queryMetrics; private Throwable initError; + private NodeResource allocation; + private final List diagnostics = new ArrayList(); @@ -126,7 +125,7 @@ public class QueryMasterTask extends CompositeService { public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext, QueryId queryId, Session session, QueryContext queryContext, - String jsonExpr, AsyncDispatcher dispatcher) { + String jsonExpr, NodeResource allocation, AsyncDispatcher dispatcher) { super(QueryMasterTask.class.getName()); this.queryMasterContext = queryMasterContext; @@ -134,82 +133,77 @@ public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext, this.session = session; this.queryContext = queryContext; this.jsonExpr = jsonExpr; + this.allocation = allocation; this.querySubmitTime = System.currentTimeMillis(); this.dispatcher = dispatcher; } public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext, QueryId queryId, Session session, QueryContext queryContext, - String jsonExpr) { - this(queryMasterContext, queryId, session, queryContext, jsonExpr, new AsyncDispatcher()); + String jsonExpr, + NodeResource allocation) { + this(queryMasterContext, queryId, session, queryContext, jsonExpr, allocation, new AsyncDispatcher()); } @Override - public void init(Configuration conf) { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("conf should be a TajoConf type."); - } - systemConf = (TajoConf)conf; + public void serviceInit(Configuration conf) throws Exception { - try { - queryTaskContext = new QueryMasterTaskContext(); + systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); - addService(dispatcher); + queryTaskContext = new QueryMasterTaskContext(); - dispatcher.register(StageEventType.class, new StageEventDispatcher()); - dispatcher.register(TaskEventType.class, new TaskEventDispatcher()); - dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher()); - dispatcher.register(QueryMasterQueryCompletedEvent.EventType.class, new QueryFinishEventHandler()); - dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher()); - dispatcher.register(LocalTaskEventType.class, new LocalTaskEventHandler()); + addService(dispatcher); - initStagingDir(); + dispatcher.register(StageEventType.class, new StageEventDispatcher()); + dispatcher.register(TaskEventType.class, new TaskEventDispatcher()); + dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher()); + dispatcher.register(QueryMasterQueryCompletedEvent.EventType.class, new QueryFinishEventHandler()); + dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher()); + dispatcher.register(LocalTaskEventType.class, new LocalTaskEventHandler()); + try { + initStagingDir(); queryMetrics = new TajoMetrics(queryId.toString()); - - super.init(systemConf); } catch (Throwable t) { LOG.error(t.getMessage(), t); initError = t; } + super.serviceInit(systemConf); } public boolean isStopped() { - return stopped.get(); + return isStopped; } @Override - public void start() { + public void serviceStart() throws Exception { startQuery(); List workersProto = queryMasterContext.getQueryMaster().getAllWorker(); for (TajoProtos.WorkerConnectionInfoProto worker : workersProto) { workerMap.put(worker.getId(), new WorkerConnectionInfo(worker)); } - super.start(); + super.serviceStart(); } @Override - public void stop() { - - if(stopped.getAndSet(true)) { - return; - } + public void serviceStop() throws Exception { + isStopped = true; LOG.info("Stopping QueryMasterTask:" + queryId); + //release QM resource getQueryTaskContext().getQueryMasterContext().getWorkerContext(). - getNodeResourceManager().getDispatcher().getEventHandler().handle(new NodeResourceDeallocateEvent(NodeResources.createResource(512).getProto())); -// try { -// resourceAllocator.stop(); -// } catch (Throwable t) { -// LOG.fatal(t.getMessage(), t); -// } + getNodeResourceManager().getDispatcher().getEventHandler().handle(new NodeResourceDeallocateEvent(allocation)); + + if (!queryContext.getBool(SessionVars.DEBUG_ENABLED)) { + cleanupQuery(getQueryId()); + } if (queryMetrics != null) { queryMetrics.report(new MetricsConsoleReporter()); } - super.stop(); + super.serviceStop(); LOG.info("Stopped QueryMasterTask:" + queryId); } //FIXME remove @@ -276,33 +270,47 @@ public void handle(TaskSchedulerEvent event) { } } + /** + * It sends a kill RPC request to a corresponding worker. + * + * @param workerId worker unique Id. + * @param taskAttemptId The TaskAttemptId to be killed. + */ + protected void killTaskAttempt(int workerId, TaskAttemptId taskAttemptId) { + NettyClientBase tajoWorkerRpc; + ExecutionBlockId ebId = taskAttemptId.getTaskId().getExecutionBlockId(); + InetSocketAddress workerAddress = getQuery().getStage(ebId).getWorkerMap().get(workerId); + + try { + tajoWorkerRpc = RpcClientManager.getInstance().getClient(workerAddress, TajoWorkerProtocol.class, true); + TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); + tajoWorkerRpcClient.killTaskAttempt(null, taskAttemptId.getProto(), NullCallback.get()); + } catch (Exception e) { + /* Worker RPC failure */ + LOG.error(e.getMessage(), e); + queryMasterContext.getEventHandler().handle(new TaskFatalErrorEvent(taskAttemptId, e.getMessage())); + } + } + private class LocalTaskEventHandler implements EventHandler { @Override - public void handle(LocalTaskEvent event) { -// TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId()); -// if (proxy != null) { -// proxy.killTaskAttempt(event.getTaskAttemptId()); -// } + public void handle(final LocalTaskEvent event) { + queryMasterContext.getEventExecutor().submit(new Runnable() { + @Override + public void run() { + killTaskAttempt(event.getWorkerId(), event.getTaskAttemptId()); + } + }); } } private class QueryFinishEventHandler implements EventHandler { + @Override public void handle(QueryMasterQueryCompletedEvent event) { - QueryId queryId = event.getQueryId(); - LOG.info("Query completion notified from " + queryId); - - while (!isTerminatedState(query.getSynchronizedState())) { - try { - synchronized (this) { - wait(10); - } - } catch (InterruptedException e) { - LOG.error(e); - } - } - LOG.info("Query final state: " + query.getSynchronizedState()); + QueryId queryId = event.getQueryId(); + LOG.info("Query completion notified from " + queryId + " final state: " + query.getSynchronizedState()); queryMasterContext.getEventHandler().handle(new QueryStopEvent(queryId)); } } @@ -535,6 +543,29 @@ public long getQuerySubmitTime() { return this.querySubmitTime; } + private void cleanupQuery(final QueryId queryId) { + Set workers = Sets.newHashSet(); + for (Stage stage : getQuery().getStages()) { + workers.addAll(stage.getWorkerMap().values()); + } + + LOG.info("Cleanup resources of all workers. Query: " + queryId + ", workers: " + workers.size()); + for (final InetSocketAddress worker : workers) { + queryMasterContext.getEventExecutor().submit(new Runnable() { + @Override + public void run() { + try { + AsyncRpcClient rpc = RpcClientManager.getInstance().getClient(worker, TajoWorkerProtocol.class, true); + TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); + tajoWorkerProtocolService.cleanup(null, queryId.getProto(), NullCallback.get()); + } catch (Throwable e) { + LOG.error(e.getMessage(), e); + } + } + }); + } + } + public class QueryMasterTaskContext { EventHandler eventHandler; public QueryMaster.QueryMasterContext getQueryMasterContext() { @@ -595,14 +626,13 @@ public float getProgress() { return query.getProgress(); } -// public AbstractResourceAllocator getResourceAllocator() { -// return resourceAllocator; -// } - public TajoMetrics getQueryMetrics() { return queryMetrics; } + /** + * A key is worker id, and a value is a worker connection information. + */ public ConcurrentMap getWorkerMap() { return workerMap; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 8a094077a7..520383608a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -20,17 +20,12 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import org.apache.commons.lang.exception.ExceptionUtils; +import com.google.common.collect.Maps; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.*; -import org.apache.hadoop.yarn.util.Records; import org.apache.tajo.*; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Schema; @@ -40,7 +35,6 @@ import org.apache.tajo.catalog.statistics.StatisticsUtil; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.planner.PhysicalPlannerImpl; import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.engine.planner.global.DataChannel; @@ -50,19 +44,13 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty; import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; -import org.apache.tajo.master.LaunchTaskRunnersEvent; -import org.apache.tajo.master.TaskRunnerGroupEvent; -import org.apache.tajo.master.TaskRunnerGroupEvent.EventType; import org.apache.tajo.master.TaskState; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.container.TajoContainer; -import org.apache.tajo.master.container.TajoContainerId; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.querymaster.Task.IntermediateEntry; -import org.apache.tajo.rpc.NettyClientBase; +import org.apache.tajo.rpc.AsyncRpcClient; import org.apache.tajo.rpc.NullCallback; import org.apache.tajo.rpc.RpcClientManager; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; @@ -79,7 +67,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.*; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; @@ -115,16 +102,12 @@ public class Stage implements EventHandler { private volatile long lastContactTime; private Thread timeoutChecker; - volatile Map tasks = new ConcurrentHashMap(); - volatile Map containers = new ConcurrentHashMap(); + private final Map tasks = Maps.newConcurrentMap(); + private final Map workerMap = Maps.newConcurrentMap(); private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition(); private static final InternalErrorTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition(); - private static final ContainerLaunchTransition CONTAINER_LAUNCH_TRANSITION = new ContainerLaunchTransition(); private static final TaskCompletedTransition TASK_COMPLETED_TRANSITION = new TaskCompletedTransition(); - private static final AllocatedContainersCancelTransition CONTAINERS_CANCEL_TRANSITION = - new AllocatedContainersCancelTransition(); private static final StageCompleteTransition STAGE_COMPLETED_TRANSITION = new StageCompleteTransition(); private static final StageFinalizeTransition STAGE_FINALIZE_TRANSITION = new StageFinalizeTransition(); private StateMachine stateMachine; @@ -150,8 +133,7 @@ public class Stage implements EventHandler { // Transitions from INITED state .addTransition(StageState.INITED, StageState.RUNNING, - StageEventType.SQ_CONTAINER_ALLOCATED, - CONTAINER_LAUNCH_TRANSITION) + StageEventType.SQ_START) .addTransition(StageState.INITED, StageState.INITED, StageEventType.SQ_DIAGNOSTIC_UPDATE, DIAGNOSTIC_UPDATE_TRANSITION) @@ -161,10 +143,7 @@ StageEventType.SQ_KILL, new KillTasksTransition()) StageEventType.SQ_INTERNAL_ERROR, INTERNAL_ERROR_TRANSITION) - // Transitions from RUNNING state - .addTransition(StageState.RUNNING, StageState.RUNNING, - StageEventType.SQ_CONTAINER_ALLOCATED, - CONTAINER_LAUNCH_TRANSITION) + // Transitions from RUNNING state .addTransition(StageState.RUNNING, StageState.RUNNING, StageEventType.SQ_TASK_COMPLETED, TASK_COMPLETED_TRANSITION) @@ -193,10 +172,8 @@ StageEventType.SQ_KILL, new KillTasksTransition()) // Transitions from KILL_WAIT state .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT, - EnumSet.of(StageEventType.SQ_START, StageEventType.SQ_CONTAINER_ALLOCATED), - CONTAINERS_CANCEL_TRANSITION) - .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT, - EnumSet.of(StageEventType.SQ_KILL), new KillTasksTransition()) + EnumSet.of(StageEventType.SQ_KILL), + new KillTasksTransition()) .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT, StageEventType.SQ_TASK_COMPLETED, TASK_COMPLETED_TRANSITION) @@ -213,6 +190,9 @@ StageEventType.SQ_KILL, new KillTasksTransition()) .addTransition(StageState.KILL_WAIT, StageState.ERROR, StageEventType.SQ_INTERNAL_ERROR, INTERNAL_ERROR_TRANSITION) + // Ignore-able events + .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT, + EnumSet.of(StageEventType.SQ_START)) // Transitions from FINALIZING state .addTransition(StageState.FINALIZING, StageState.FINALIZING, @@ -233,9 +213,6 @@ StageEventType.SQ_KILL, new KillTasksTransition()) StageEventType.SQ_KILL) // Transitions from SUCCEEDED state - .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED, - StageEventType.SQ_CONTAINER_ALLOCATED, - CONTAINERS_CANCEL_TRANSITION) .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED, StageEventType.SQ_DIAGNOSTIC_UPDATE, DIAGNOSTIC_UPDATE_TRANSITION) @@ -247,13 +224,9 @@ StageEventType.SQ_KILL, new KillTasksTransition()) EnumSet.of( StageEventType.SQ_START, StageEventType.SQ_KILL, - StageEventType.SQ_CONTAINER_ALLOCATED, StageEventType.SQ_SHUFFLE_REPORT)) // Transitions from KILLED state - .addTransition(StageState.KILLED, StageState.KILLED, - StageEventType.SQ_CONTAINER_ALLOCATED, - CONTAINERS_CANCEL_TRANSITION) .addTransition(StageState.KILLED, StageState.KILLED, StageEventType.SQ_DIAGNOSTIC_UPDATE, DIAGNOSTIC_UPDATE_TRANSITION) @@ -265,15 +238,11 @@ StageEventType.SQ_KILL, new KillTasksTransition()) EnumSet.of( StageEventType.SQ_START, StageEventType.SQ_KILL, - StageEventType.SQ_CONTAINER_ALLOCATED, StageEventType.SQ_SHUFFLE_REPORT, StageEventType.SQ_STAGE_COMPLETED, StageEventType.SQ_FAILED)) // Transitions from FAILED state - .addTransition(StageState.FAILED, StageState.FAILED, - StageEventType.SQ_CONTAINER_ALLOCATED, - CONTAINERS_CANCEL_TRANSITION) .addTransition(StageState.FAILED, StageState.FAILED, StageEventType.SQ_DIAGNOSTIC_UPDATE, DIAGNOSTIC_UPDATE_TRANSITION) @@ -285,13 +254,9 @@ StageEventType.SQ_KILL, new KillTasksTransition()) EnumSet.of( StageEventType.SQ_START, StageEventType.SQ_KILL, - StageEventType.SQ_CONTAINER_ALLOCATED, StageEventType.SQ_FAILED)) // Transitions from ERROR state - .addTransition(StageState.ERROR, StageState.ERROR, - StageEventType.SQ_CONTAINER_ALLOCATED, - CONTAINERS_CANCEL_TRANSITION) .addTransition(StageState.ERROR, StageState.ERROR, StageEventType.SQ_DIAGNOSTIC_UPDATE, DIAGNOSTIC_UPDATE_TRANSITION) @@ -703,35 +668,44 @@ private TableStats[] computeStatFromTasks() { } private void stopScheduler() { - // If there are launched TaskRunners, send the 'shouldDie' message to all r - // via received task requests. if (taskScheduler != null) { taskScheduler.stop(); } } - - //////////////FIXME - public void stopExecutionBlock(TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto) { - - for (WorkerConnectionInfo worker : getContext().getWorkerMap().values()) { - NettyClientBase tajoWorkerRpc = null; - try { - InetSocketAddress addr = new InetSocketAddress(worker.getHost(), worker.getPeerRpcPort()); - tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); - TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); - - tajoWorkerRpcClient.stopExecutionBlock(null, requestProto, NullCallback.get(PrimitiveProtos.BoolProto.class)); - } catch (Throwable e) { - LOG.error(e.getMessage(), e); - } + /** + * Get the task launched worker + */ + protected Map getWorkerMap() { + return workerMap; + } + + private void sendStopExecutionBlockEvent(final TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto) { + + for (final InetSocketAddress worker : getWorkerMap().values()) { + getContext().getQueryMasterContext().getEventExecutor().submit(new Runnable() { + @Override + public void run() { + try { + AsyncRpcClient tajoWorkerRpc = + RpcClientManager.getInstance().getClient(worker, TajoWorkerProtocol.class, true); + TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); + tajoWorkerRpcClient.stopExecutionBlock(null, + requestProto, NullCallback.get(PrimitiveProtos.BoolProto.class)); + } catch (Throwable e) { + LOG.error(e.getMessage(), e); + } + } + }); } } - private void releaseContainers() { - // If there are still live TaskRunners, try to kill the containers. and send the shuffle report request + /** + * Sends stopping request to all worker + */ + protected void stopExecutionBlock() { + // If there are still live tasks, try to kill the tasks. and send the shuffle report request - //eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), containers.values())); List ebIds = Lists.newArrayList(); if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) { List childs = getMasterPlan().getChilds(getId()); @@ -739,16 +713,17 @@ private void releaseContainers() { for (ExecutionBlock executionBlock : childs) { ebIds.add(executionBlock.getId().getProto()); } - - //getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds); } - TajoWorkerProtocol.StopExecutionBlockRequestProto.Builder stopRequest = TajoWorkerProtocol.StopExecutionBlockRequestProto.newBuilder(); - TajoWorkerProtocol.ExecutionBlockListProto.Builder builder = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); - builder.addAllExecutionBlockId(Lists.newArrayList(ebIds)); - stopRequest.setChild(builder.build()); + TajoWorkerProtocol.StopExecutionBlockRequestProto.Builder + stopRequest = TajoWorkerProtocol.StopExecutionBlockRequestProto.newBuilder(); + TajoWorkerProtocol.ExecutionBlockListProto.Builder + cleanupList = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); + + cleanupList.addAllExecutionBlockId(Lists.newArrayList(ebIds)); + stopRequest.setChild(cleanupList.build()); stopRequest.setExecutionBlockId(getId().getProto()); - stopExecutionBlock(stopRequest.build()); + sendStopExecutionBlockEvent(stopRequest.build()); } /** @@ -853,7 +828,7 @@ public void run() { } else { if(stage.getSynchronizedState() == StageState.INITED) { stage.taskScheduler.start(); - allocateContainers(stage); + stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_START)); } else { /* all tasks are killed before stage are inited */ if (stage.getTotalScheduledObjectsCount() == stage.getCompletedTaskCount()) { @@ -1091,38 +1066,6 @@ public static long getInputVolume(MasterPlan masterPlan, QueryMasterTask.QueryMa } } - public static void allocateContainers(Stage stage) { - stage.getEventHandler().handle(new StageContainerAllocationEvent(stage.getId(), new ArrayList())); - return; - - /*ExecutionBlock execBlock = stage.getBlock(); - - //TODO consider disk slot - int requiredMemoryMBPerTask = 512; - -// int numRequest = stage.getContext().getResourceAllocator().calculateNumRequestContainers( -// stage.getContext().getQueryMasterContext().getWorkerContext(), -// stage.schedulerContext.getEstimatedTaskNum(), -// requiredMemoryMBPerTask -// ); - - int numRequest = 0; - - final Resource resource = Records.newRecord(Resource.class); - - resource.setMemory(requiredMemoryMBPerTask); - - LOG.info("Request Container for " + stage.getId() + " containers=" + numRequest); - - Priority priority = Records.newRecord(Priority.class); - priority.setPriority(stage.getPriority()); - ContainerAllocationEvent event = - new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ, - stage.getId(), priority, resource, numRequest, - stage.masterPlan.isLeaf(execBlock), 0.0f); - stage.eventHandler.handle(event);*/ - } - private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException { ExecutionBlock execBlock = stage.getBlock(); ScanNode[] scans = execBlock.getScanNodes(); @@ -1206,69 +1149,6 @@ public static Task newEmptyTask(TaskSchedulerContext schedulerContext, return unit; } - private static class ContainerLaunchTransition - implements SingleArcTransition { - - @Override - public void transition(Stage stage, StageEvent event) { - if (!(event instanceof StageContainerAllocationEvent)) { - throw new IllegalArgumentException("event should be a StageContainerAllocationEvent type."); - } - try { - StageContainerAllocationEvent allocationEvent = - (StageContainerAllocationEvent) event; - for (TajoContainer container : allocationEvent.getAllocatedContainer()) { - TajoContainerId cId = container.getId(); - if (stage.containers.containsKey(cId)) { - stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), - "Duplicated containers are allocated: " + cId.toString())); - stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR)); - } - stage.containers.put(cId, container); - } - LOG.info("Stage (" + stage.getId() + ") has " + stage.containers.size() + " containers!"); -// stage.eventHandler.handle( -// new LaunchTaskRunnersEvent(stage.getId(), allocationEvent.getAllocatedContainer(), -// stage.getContext().getQueryContext(), -// CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) -// ); - - stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_START)); - } catch (Throwable t) { - stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), - ExceptionUtils.getStackTrace(t))); - stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR)); - } - } - } - - /** - * It is used in KILL_WAIT state against Contained Allocated event. - * It just returns allocated containers to resource manager. - */ - private static class AllocatedContainersCancelTransition implements SingleArcTransition { - @Override - public void transition(Stage stage, StageEvent event) { - if (!(event instanceof StageContainerAllocationEvent)) { - throw new IllegalArgumentException("event should be a StageContainerAllocationEvent type."); - } - try { - StageContainerAllocationEvent allocationEvent = - (StageContainerAllocationEvent) event; - stage.eventHandler.handle( - new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, - stage.getId(), allocationEvent.getAllocatedContainer())); - LOG.info(String.format("[%s] %d allocated containers are canceled", - stage.getId().toString(), - allocationEvent.getAllocatedContainer().size())); - } catch (Throwable t) { - stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), - ExceptionUtils.getStackTrace(t))); - stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR)); - } - } - } - private static class TaskCompletedTransition implements SingleArcTransition { @Override @@ -1330,7 +1210,7 @@ public void transition(Stage stage, StageEvent stageEvent) { private void cleanup() { stopScheduler(); - releaseContainers(); + stopExecutionBlock(); this.finalStageHistory = makeStageHistory(); this.finalStageHistory.setTasks(makeTaskHistories()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java index c1b9273ede..cfce0c9ac0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java @@ -396,7 +396,8 @@ private static class KillTaskTransition implements SingleArcTransition workerInfoMap = Maps.newConcurrentMap(); - - public WorkerConnectionInfo getWorkerConnectionInfo(int workerId) { - return workerInfoMap.get(workerId); - } - - public void addWorkerConnectionInfo(WorkerConnectionInfo connectionInfo) { - workerInfoMap.putIfAbsent(connectionInfo.getId(), connectionInfo); - } - - private Map containers = Maps.newConcurrentMap(); - - public AbstractResourceAllocator() { - super(AbstractResourceAllocator.class.getName()); - } - - public void addContainer(TajoContainerId cId, ContainerProxy container) { - containers.put(cId, container); - } - - public void removeContainer(TajoContainerId cId) { - containers.remove(cId); - } - - public boolean containsContainer(TajoContainerId cId) { - return containers.containsKey(cId); - } - - public ContainerProxy getContainer(TajoContainerId cId) { - return containers.get(cId); - } - - public Map getContainers() { - return containers; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index 9156ae6206..3ba4ef7c04 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -141,17 +141,18 @@ private void release(NodeResource resource) { } private NodeResource createWorkerResource(TajoConf conf) { - int memoryMb; - if (conf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) { - memoryMb = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); - } else { - memoryMb = Math.min((int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB), - conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB)); + int memoryMb = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); + if (!conf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) { + // Set memory resource to max heap + int maxHeap = (int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB); + if(maxHeap > memoryMb) { + memoryMb = maxHeap; + } } int vCores = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); - int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM); + int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS); int dataNodeStorageSize = DiskUtil.getDataNodeStorageSize(); if (conf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && dataNodeStorageSize > 0) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index a488108096..edd6ed67c2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -27,10 +27,12 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.resource.DefaultResourceCalculator; -import org.apache.tajo.resource.NodeResource; -import org.apache.tajo.rpc.*; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.AsyncRpcClient; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.rpc.RpcClientManager; +import org.apache.tajo.rpc.RpcConstants; import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.worker.event.NodeStatusEvent; @@ -43,7 +45,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; - import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; /** @@ -62,7 +63,7 @@ public class NodeStatusUpdater extends AbstractService implements EventHandler events = Lists.newArrayList(); try { /* batch update to ResourceTracker */ - drain(events, Math.max(queueingLimit, 1), heartBeatInterval, TimeUnit.MILLISECONDS); + drain(events, Math.max(queueingThreshold, 1), heartBeatInterval, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { break; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java deleted file mode 100644 index 62db0a44ab..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java +++ /dev/null @@ -1,413 +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.worker; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -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.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; -import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.master.*; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.container.TajoContainer; -import org.apache.tajo.master.container.TajoContainerId; -import org.apache.tajo.master.event.ContainerAllocationEvent; -import org.apache.tajo.master.event.ContainerAllocatorEventType; -import org.apache.tajo.master.event.StageContainerAllocationEvent; -import org.apache.tajo.master.rm.TajoWorkerContainer; -import org.apache.tajo.master.rm.TajoWorkerContainerId; -import org.apache.tajo.master.rm.Worker; -import org.apache.tajo.master.rm.WorkerResource; -import org.apache.tajo.querymaster.QueryMasterTask; -import org.apache.tajo.querymaster.Stage; -import org.apache.tajo.querymaster.StageState; -import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NettyClientBase; -import org.apache.tajo.rpc.NullCallback; -import org.apache.tajo.rpc.RpcClientManager; -import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; -import org.apache.tajo.service.ServiceTracker; -import org.apache.tajo.util.ApplicationIdUtils; - -import java.net.InetSocketAddress; -import java.util.*; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicBoolean; - -public class TajoResourceAllocator extends AbstractResourceAllocator { - private static final Log LOG = LogFactory.getLog(TajoResourceAllocator.class); - - private TajoConf tajoConf; - private QueryMasterTask.QueryMasterTaskContext queryTaskContext; - private final ExecutorService allocationExecutor; - private final Deallocator deallocator; - - private AtomicBoolean stopped = new AtomicBoolean(false); - - public TajoResourceAllocator(QueryMasterTask.QueryMasterTaskContext queryTaskContext) { - this.queryTaskContext = queryTaskContext; - allocationExecutor = Executors.newFixedThreadPool( - queryTaskContext.getConf().getIntVar(TajoConf.ConfVars.YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM)); - deallocator = new Deallocator(); - } - - @Override - public TajoContainerId makeContainerId(ContainerProtocol.TajoContainerIdProto containerIdProto) { - TajoWorkerContainerId containerId = new TajoWorkerContainerId(); - ApplicationAttemptId appAttemptId = new ApplicationAttemptIdPBImpl(containerIdProto.getAppAttemptId()); - containerId.setApplicationAttemptId(appAttemptId); - containerId.setId(containerIdProto.getId()); - return containerId; - } - - @Override - public void allocateTaskWorker() { - } - - @Override - public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext, - int numTasks, - int memoryMBPerTask) { - //TODO consider disk slot - -// ClusterResourceSummary clusterResource = workerContext.getClusterResource(); -// int clusterSlots = clusterResource == null ? 0 : clusterResource.getTotalMemoryMB() / memoryMBPerTask; -// clusterSlots = Math.max(1, clusterSlots - 1); // reserve query master slot -// LOG.info("CalculateNumberRequestContainer - Number of Tasks=" + numTasks + -// ", Number of Cluster Slots=" + clusterSlots); -// return Math.min(numTasks, clusterSlots); - return 0; - } - - @Override - public void init(Configuration conf) { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("conf should be a TajoConf type."); - } - tajoConf = (TajoConf)conf; - - queryTaskContext.getDispatcher().register(TaskRunnerGroupEvent.EventType.class, new TajoTaskRunnerLauncher()); - - queryTaskContext.getDispatcher().register(ContainerAllocatorEventType.class, new TajoWorkerAllocationHandler()); - - deallocator.start(); - - super.init(conf); - } - - @Override - public synchronized void stop() { - if (stopped.compareAndSet(false, true)) { - return; - } - - allocationExecutor.shutdownNow(); - deallocator.shutdown(); - -// Map containers = queryTaskContext.getResourceAllocator() -// .getContainers(); -// List list = new ArrayList(containers.values()); -// for(ContainerProxy eachProxy: list) { -// try { -// eachProxy.stopContainer(); -// } catch (Throwable e) { -// LOG.warn(e.getMessage(), e); -// } -// } - - workerInfoMap.clear(); - super.stop(); - } - - @Override - public void start() { - super.start(); - } - - class TajoTaskRunnerLauncher implements TaskRunnerLauncher { - @Override - public void handle(TaskRunnerGroupEvent event) { - if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_LAUNCH) { - if (!(event instanceof LaunchTaskRunnersEvent)) { - throw new IllegalArgumentException("event should be a LaunchTaskRunnersEvent type."); - } - LaunchTaskRunnersEvent launchEvent = (LaunchTaskRunnersEvent) event; - launchTaskRunners(launchEvent); - } else if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_CLEANUP) { - stopContainers(event.getContainers()); - stopExecutionBlock(event.getExecutionBlockId(), event.getContainers()); - } - } - } - - private void launchTaskRunners(LaunchTaskRunnersEvent event) { - // Query in standby mode doesn't need launch Worker. - // But, Assign ExecutionBlock to assigned tajo worker - for(TajoContainer eachContainer: event.getContainers()) { - TajoContainerProxy containerProxy = new TajoContainerProxy(queryTaskContext, tajoConf, - eachContainer, event.getQueryContext(), event.getExecutionBlockId(), event.getPlanJson()); - allocationExecutor.submit(new LaunchRunner(eachContainer.getId(), containerProxy)); - } - } - - public void stopExecutionBlock(final ExecutionBlockId executionBlockId, - Collection containers) { - Set workers = Sets.newHashSet(); - for (TajoContainer container : containers){ - workers.add(container.getNodeId()); - } - - for (final NodeId worker : workers) { - allocationExecutor.submit(new Runnable() { - @Override - public void run() { - stopExecutionBlock(executionBlockId, worker); - } - }); - } - } - - private void stopExecutionBlock(ExecutionBlockId executionBlockId, NodeId worker) { -// NettyClientBase tajoWorkerRpc = null; -// try { -// InetSocketAddress addr = new InetSocketAddress(worker.getHost(), worker.getPort()); -// tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); -// TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); -// -// tajoWorkerRpcClient.stopExecutionBlock(null, executionBlockId.getProto(), -// NullCallback.get(PrimitiveProtos.BoolProto.class)); -// } catch (Throwable e) { -// LOG.error(e.getMessage(), e); -// } - } - - protected static class LaunchRunner implements Runnable { - private final ContainerProxy proxy; - private final TajoContainerId id; - public LaunchRunner(TajoContainerId id, ContainerProxy proxy) { - this.proxy = proxy; - this.id = id; - } - @Override - public void run() { - proxy.launch(null); - if (LOG.isDebugEnabled()) { - LOG.debug("ContainerProxy started:" + id); - } - } - } - - private void stopContainers(Collection containers) { - deallocator.submit(Iterables.transform(containers, new Function() { - public TajoContainerId apply(TajoContainer input) { return input.getId(); } - })); - } - - private static final TajoContainerId FIN = new TajoWorkerContainerId(); - - private class Deallocator extends Thread { - - private final BlockingDeque queue = new LinkedBlockingDeque(); - - public Deallocator() { - setName("Deallocator"); - setDaemon(true); - } - - private void submit(Iterable container) { - queue.addAll(Lists.newArrayList(container)); - } - - private void shutdown() { - queue.add(FIN); - } - - @Override - public void run() { - //final AbstractResourceAllocator allocator = queryTaskContext.getResourceAllocator(); - /*while (!stopped.get() || !queue.isEmpty()) { - TajoContainerId containerId; - try { - containerId = queue.take(); - } catch (InterruptedException e) { - continue; - } - if (containerId == FIN) { - break; - } - ContainerProxy proxy = allocator.getContainer(containerId); - if (proxy == null) { - continue; - } - try { - LOG.info("Stopping ContainerProxy: " + proxy.getContainerId() + "," + proxy.getBlockId()); - proxy.stopContainer(); - } catch (Exception e) { - LOG.warn("Failed to stop container " + proxy.getContainerId() + "," + proxy.getBlockId(), e); - } - }*/ - LOG.info("Deallocator exiting"); - } - } - - class TajoWorkerAllocationHandler implements EventHandler { - @Override - public void handle(ContainerAllocationEvent event) { - allocationExecutor.submit(new TajoWorkerAllocationThread(event)); - } - } - - class TajoWorkerAllocationThread extends Thread { - ContainerAllocationEvent event; - TajoWorkerAllocationThread(ContainerAllocationEvent event) { - this.event = event; - } - - @Override - public void run() { - LOG.info("Start TajoWorkerAllocationThread"); -// CallFuture callBack = -// new CallFuture(); - - //TODO consider task's resource usage pattern - int requiredMemoryMB = tajoConf.getIntVar(TajoConf.ConfVars.TASK_DEFAULT_MEMORY); - float requiredDiskSlots = tajoConf.getFloatVar(TajoConf.ConfVars.TASK_DEFAULT_DISK); - -// WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder() -// .setMinMemoryMBPerContainer(requiredMemoryMB) -// .setMaxMemoryMBPerContainer(requiredMemoryMB) -// .setNumContainers(event.getRequiredNum()) -// .setResourceRequestPriority(!event.isLeafQuery() ? -// ResourceRequestPriority.MEMORY : ResourceRequestPriority.DISK) -// .setMinDiskSlotPerContainer(requiredDiskSlots) -// .setMaxDiskSlotPerContainer(requiredDiskSlots) -// .setQueryId(event.getExecutionBlockId().getQueryId().getProto()) -// .build(); -// -// -// NettyClientBase tmClient = null; -// try { -// ServiceTracker serviceTracker = queryTaskContext.getQueryMasterContext().getWorkerContext().getServiceTracker(); -// tmClient = RpcClientManager.getInstance(). -// getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); -// QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); -// masterClientService.allocateWorkerResources(callBack.getController(), request, callBack); -// } catch (Throwable e) { -// LOG.error(e.getMessage(), e); -// } -// -// WorkerResourceAllocationResponse response = null; -// while(!stopped.get()) { -// try { -// response = callBack.get(3, TimeUnit.SECONDS); -// break; -// } catch (InterruptedException e) { -// if(stopped.get()) { -// return; -// } -// } catch (TimeoutException e) { -// LOG.info("No available worker resource for " + event.getExecutionBlockId()); -// continue; -// } catch (ExecutionException e) { -// LOG.error(e.getMessage(), e); -// break; -// } -// } -// -// int numAllocatedContainers = 0; -// -// if(response != null) { -// List allocatedResources = response.getWorkerAllocatedResourceList(); -// ExecutionBlockId executionBlockId = event.getExecutionBlockId(); -// -// List containers = new ArrayList(); -// for(WorkerAllocatedResource eachAllocatedResource: allocatedResources) { -// TajoWorkerContainer container = new TajoWorkerContainer(); -// NodeId nodeId = NodeId.newInstance(eachAllocatedResource.getConnectionInfo().getHost(), -// eachAllocatedResource.getConnectionInfo().getPeerRpcPort()); -// -// TajoWorkerContainerId containerId = new TajoWorkerContainerId(); -// -// containerId.setApplicationAttemptId( -// ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId(), -// eachAllocatedResource.getContainerId().getAppAttemptId().getAttemptId())); -// containerId.setId(eachAllocatedResource.getContainerId().getId()); -// -// container.setId(containerId); -// container.setNodeId(nodeId); -// -// -// WorkerResource workerResource = new WorkerResource(); -// workerResource.setMemoryMB(eachAllocatedResource.getAllocatedMemoryMB()); -// workerResource.setDiskSlots(eachAllocatedResource.getAllocatedDiskSlots()); -// -// Worker worker = new Worker(null, workerResource, -// new WorkerConnectionInfo(eachAllocatedResource.getConnectionInfo())); -// container.setWorkerResource(worker); -// addWorkerConnectionInfo(worker.getConnectionInfo()); -// containers.add(container); -// } -// -// StageState state = queryTaskContext.getStage(executionBlockId).getSynchronizedState(); -// if (!Stage.isRunningState(state)) { -// List containerIds = new ArrayList(); -// for(TajoContainer eachContainer: containers) { -// containerIds.add(eachContainer.getId()); -// } -// try { -// TajoContainerProxy.releaseWorkerResource(queryTaskContext, executionBlockId, containerIds); -// } catch (Throwable e) { -// deallocator.submit(containerIds); -// LOG.error(e.getMessage(), e); -// } -// return; -// } -// -// if (allocatedResources.size() > 0) { -// if(LOG.isDebugEnabled()) { -// LOG.debug("StageContainerAllocationEvent fire:" + executionBlockId); -// } -// queryTaskContext.getEventHandler().handle(new StageContainerAllocationEvent(executionBlockId, containers)); -// } -// numAllocatedContainers += allocatedResources.size(); -// -// } -// if(event.getRequiredNum() > numAllocatedContainers) { -// ContainerAllocationEvent shortRequestEvent = new ContainerAllocationEvent( -// event.getType(), event.getExecutionBlockId(), event.getPriority(), -// event.getResource(), -// event.getRequiredNum() - numAllocatedContainers, -// event.isLeafQuery(), event.getProgress() -// ); -// queryTaskContext.getEventHandler().handle(shortRequestEvent); -// -// } -// LOG.info("Stop TajoWorkerAllocationThread"); - } - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 7b87741a45..da3ce30730 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -68,7 +68,6 @@ import java.util.HashMap; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import static org.apache.tajo.conf.TajoConf.ConfVars; @@ -145,12 +144,9 @@ public void startWorker(TajoConf systemConf, String[] args) { @Override public void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("conf should be a TajoConf type."); - } Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook())); - this.systemConf = (TajoConf)conf; + this.systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); RackResolver.init(systemConf); RpcClientManager rpcManager = RpcClientManager.getInstance(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index 85b769634f..ff3a09dfe1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -18,7 +18,6 @@ package org.apache.tajo.worker; -import com.google.common.base.Preconditions; import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; import org.apache.commons.logging.Log; @@ -27,13 +26,14 @@ import org.apache.hadoop.service.CompositeService; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryId; -import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TajoIdProtos; +import org.apache.tajo.TaskAttemptId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.event.ExecutionBlockStopEvent; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; @@ -56,9 +56,9 @@ public TajoWorkerManagerService(TajoWorker.WorkerContext workerContext, int port } @Override - public void init(Configuration conf) { - Preconditions.checkArgument(conf instanceof TajoConf); - TajoConf tajoConf = (TajoConf) conf; + public void serviceInit(Configuration conf) throws Exception { + + TajoConf tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); try { // Setup RPC server InetSocketAddress initIsa = @@ -79,21 +79,16 @@ public void init(Configuration conf) { // Get the master address LOG.info("TajoWorkerManagerService is bind to " + bindAddr); tajoConf.setVar(TajoConf.ConfVars.WORKER_PEER_RPC_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddr)); - super.init(tajoConf); - } - - @Override - public void start() { - super.start(); + super.serviceInit(tajoConf); } @Override - public void stop() { + public void serviceStop() throws Exception { if(rpcServer != null) { rpcServer.shutdown(); } LOG.info("TajoWorkerManagerService stopped"); - super.stop(); + super.serviceStop(); } public InetSocketAddress getBindAddr() { @@ -111,27 +106,11 @@ public void ping(RpcController controller, public void allocateTasks(RpcController controller, TajoWorkerProtocol.BatchAllocationRequestProto request, RpcCallback done) { - //LOG.info("Try to allocate Tasks : " + request.toString()); + workerContext.getWorkerSystemMetrics().counter("query", "allocationRequestNum").inc(); workerContext.getNodeResourceManager().getDispatcher(). getEventHandler().handle(new NodeResourceAllocateEvent(request, done)); } -// @Override -// public void startExecutionBlock(RpcController controller, -// TajoWorkerProtocol.StartExecutionBlockRequestProto request, -// RpcCallback done) { -// workerContext.getWorkerSystemMetrics().counter("query", "executedExecutionBlocksNum").inc(); -// -// try { -// workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new TaskRunnerStartEvent(request)); -// done.run(TajoWorker.TRUE_PROTO); -// } catch (Throwable t) { -// LOG.error(t.getMessage(), t); -// controller.setFailed(t.getMessage()); -// done.run(TajoWorker.FALSE_PROTO); -// } -// } - @Override public void stopExecutionBlock(RpcController controller, TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto, @@ -165,18 +144,4 @@ public void cleanup(RpcController controller, TajoIdProtos.QueryIdProto request, workerContext.cleanup(new QueryId(request).toString()); done.run(TajoWorker.TRUE_PROTO); } - - @Deprecated - @Override - public void cleanupExecutionBlocks(RpcController controller, - TajoWorkerProtocol.ExecutionBlockListProto ebIds, - RpcCallback done) { - for (TajoIdProtos.ExecutionBlockIdProto executionBlockIdProto : ebIds.getExecutionBlockIdList()) { - String inputDir = ExecutionBlockContext.getBaseInputDir(new ExecutionBlockId(executionBlockIdProto)).toString(); - workerContext.cleanup(inputDir); - String outputDir = ExecutionBlockContext.getBaseOutputDir(new ExecutionBlockId(executionBlockIdProto)).toString(); - workerContext.cleanup(outputDir); - } - done.run(TajoWorker.TRUE_PROTO); - } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index dd21e222db..78644d11ee 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -25,18 +25,16 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.TaskId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.engine.query.TaskRequest; import org.apache.tajo.engine.query.TaskRequestImpl; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.resource.NodeResource; -import org.apache.tajo.resource.NodeResources; -import org.apache.tajo.worker.event.*; +import org.apache.tajo.util.TUtil; +import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; +import org.apache.tajo.worker.event.TaskExecutorEvent; +import org.apache.tajo.worker.event.TaskStartEvent; import java.io.IOException; import java.util.Map; @@ -68,11 +66,8 @@ public TaskExecutor(TajoWorker.WorkerContext workerContext) { @Override protected void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - this.tajoConf = (TajoConf) conf; + this.tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); this.workerContext.getTaskManager().getDispatcher().register(TaskExecutorEvent.EventType.class, this); super.serviceInit(conf); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index b767cd6363..875530b30f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -31,6 +31,7 @@ import org.apache.tajo.TaskId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.event.*; import java.io.IOException; @@ -58,11 +59,8 @@ public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext @Override protected void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - this.tajoConf = (TajoConf)conf; + this.tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); dispatcher.register(TaskManagerEvent.EventType.class, this); super.serviceInit(conf); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.java new file mode 100644 index 0000000000..50475b86cd --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.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.worker.event; + + +import com.google.protobuf.RpcCallback; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; +import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; + +public class QMResourceAllocateEvent extends NodeResourceEvent { + + private QueryCoordinatorProtocol.AllocationResourceProto request; + private RpcCallback callback; + + public QMResourceAllocateEvent(QueryCoordinatorProtocol.AllocationResourceProto request, + RpcCallback callback) { + super(EventType.QM_ALLOCATE); + this.callback = callback; + this.request = request; + } + + public QueryCoordinatorProtocol.AllocationResourceProto getRequest() { + return request; + } + + public RpcCallback getCallback() { + return callback; + } +} diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto index 22c03e6641..dc6fc692e1 100644 --- a/tajo-core/src/main/proto/QueryMasterProtocol.proto +++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto @@ -34,7 +34,6 @@ package hadoop.yarn; service QueryMasterProtocolService { //from Worker - rpc getTask(GetTaskRequestProto) returns (TaskRequestProto); rpc statusUpdate (TaskStatusProto) returns (NullProto); rpc ping (ExecutionBlockIdProto) returns (NullProto); rpc fatalError(TaskFatalErrorReport) returns (NullProto); @@ -44,5 +43,5 @@ service QueryMasterProtocolService { //from TajoMaster's QueryJobManager rpc killQuery(QueryIdProto) returns (NullProto); rpc executeQuery(QueryExecutionRequestProto) returns (NullProto); - rpc startQueryMaster(AllocationResourceProto) returns (BoolProto); + rpc allocateQueryMaster(AllocationResourceProto) returns (BoolProto); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 9386a4aca8..e210c5d85a 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -29,6 +29,7 @@ import "CatalogProtos.proto"; import "PrimitiveProtos.proto"; import "Plan.proto"; import "ContainerProtocol.proto"; +import "QueryCoordinatorProtocol.proto"; package hadoop.yarn; @@ -158,15 +159,9 @@ message QueryExecutionRequestProto { required QueryIdProto queryId = 1; required SessionProto session = 2; required KeyValueSetProto queryContext = 3; - required StringProto exprInJson = 5; - optional StringProto logicalPlanJson = 6; -} - -// deprecated -message GetTaskRequestProto { - required int32 workerId = 1; - required TajoContainerIdProto containerId = 2; - required ExecutionBlockIdProto executionBlockId = 3; + required StringProto exprInJson = 4; + optional StringProto logicalPlanJson = 5; + required AllocationResourceProto allocation = 6; } message DataChannelProto { @@ -223,12 +218,10 @@ service TajoWorkerProtocolService { rpc ping (TaskAttemptIdProto) returns (BoolProto); // from QueryMaster(Worker) - //rpc startExecutionBlock(StartExecutionBlockRequestProto) returns (BoolProto); rpc allocateTasks(BatchAllocationRequestProto) returns (BatchAllocationResponseProto); rpc stopExecutionBlock(StopExecutionBlockRequestProto) returns (BoolProto); rpc killTaskAttempt(TaskAttemptIdProto) returns (BoolProto); rpc cleanup(QueryIdProto) returns (BoolProto); - rpc cleanupExecutionBlocks(ExecutionBlockListProto) returns (BoolProto); } message EnforceProperty { diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index b8016f814a..8129ca6ebc 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -125,7 +125,7 @@ void initPropertiesAndConfigs() { conf.setClassVar(ConfVars.GLOBAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, GlobalPlanTestRuleProvider.class); conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 3072); - conf.setFloat(ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.varname, 4.0f); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 4); // Client API RPC @@ -148,7 +148,6 @@ void initPropertiesAndConfigs() { // Resource allocator conf.setIntVar(ConfVars.$QUERY_EXECUTE_PARALLEL_MAX, 10); - conf.setIntVar(ConfVars.YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM, 6); // make twice of parallel_max // Memory cache termination conf.setIntVar(ConfVars.WORKER_HISTORY_EXPIRE_PERIOD, 1); diff --git a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java index 592767ae60..de02c281e4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java @@ -47,7 +47,7 @@ private TajoResourceManager initResourceManager() throws Exception { tajoConf = new org.apache.tajo.conf.TajoConf(); // tajoConf.setFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT, 0.0f); - tajoConf.setIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB, 512); + tajoConf.setIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY, 512); tajoConf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS, "localhost:0"); TajoResourceManager tajoResourceManager = new TajoResourceManager(tajoConf); tajoResourceManager.init(tajoConf); diff --git a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java deleted file mode 100644 index db769f81d1..0000000000 --- a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java +++ /dev/null @@ -1,115 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.master.scheduler; - -import org.apache.tajo.QueryId; -import org.apache.tajo.TajoProtos; -import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.benchmark.TPCH; -import org.apache.tajo.client.TajoClient; -import org.apache.tajo.client.TajoClientUtil; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.ClientProtos; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import java.io.File; -import java.sql.ResultSet; - -import static org.junit.Assert.*; - -public class TestFifoScheduler { -// private static TajoTestingCluster cluster; -// private static TajoConf conf; -// private static TajoClient client; -// private static String query = -// "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey"; -// -// @BeforeClass -// public static void setUp() throws Exception { -// cluster = new TajoTestingCluster(); -// cluster.startMiniClusterInLocal(1); -// conf = cluster.getConfiguration(); -// client = cluster.newTajoClient(); -// File file = TPCH.getDataFile("lineitem"); -// client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) " -// + "using text location 'file://" + file.getAbsolutePath() + "'"); -// assertTrue(client.existTable("default.lineitem")); -// } -// -// @AfterClass -// public static void tearDown() throws Exception { -// if (client != null) client.close(); -// if (cluster != null) cluster.shutdownMiniCluster(); -// } -// -// @Test -// public final void testKillScheduledQuery() throws Exception { -// ClientProtos.SubmitQueryResponse res = client.executeQuery(query); -// ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query); -// QueryId queryId = new QueryId(res.getQueryId()); -// QueryId queryId2 = new QueryId(res2.getQueryId()); -// -// cluster.waitForQuerySubmitted(queryId); -// client.killQuery(queryId2); -// assertEquals(TajoProtos.QueryState.QUERY_KILLED, client.getQueryStatus(queryId2).getState()); -// } -// -// @Test -// public final void testForwardedQuery() throws Exception { -// ClientProtos.SubmitQueryResponse res = client.executeQuery(query); -// ClientProtos.SubmitQueryResponse res2 = client.executeQuery("select * from lineitem limit 1"); -// assertTrue(res.getIsForwarded()); -// assertFalse(res2.getIsForwarded()); -// -// QueryId queryId = new QueryId(res.getQueryId()); -// QueryId queryId2 = new QueryId(res2.getQueryId()); -// cluster.waitForQuerySubmitted(queryId); -// -// assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, client.getQueryStatus(queryId2).getState()); -// ResultSet resSet = TajoClientUtil.createResultSet(client, res2, 1); -// assertNotNull(resSet); -// } -// -// @Test -// public final void testScheduledQuery() throws Exception { -// ClientProtos.SubmitQueryResponse res = client.executeQuery("select sleep(1) from lineitem"); -// ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query); -// ClientProtos.SubmitQueryResponse res3 = client.executeQuery(query); -// ClientProtos.SubmitQueryResponse res4 = client.executeQuery(query); -// -// QueryId queryId = new QueryId(res.getQueryId()); -// QueryId queryId2 = new QueryId(res2.getQueryId()); -// QueryId queryId3 = new QueryId(res3.getQueryId()); -// QueryId queryId4 = new QueryId(res4.getQueryId()); -// -// cluster.waitForQuerySubmitted(queryId); -// -// assertFalse(TajoClientUtil.isQueryComplete(client.getQueryStatus(queryId).getState())); -// -// assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId2).getState()); -// assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId3).getState()); -// assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId4).getState()); -// -// client.killQuery(queryId4); -// client.killQuery(queryId3); -// client.killQuery(queryId2); -// } -} diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index ae32eb4189..5935dcfe92 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -19,7 +19,6 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Lists; -import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; @@ -35,21 +34,18 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequestImpl; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.event.*; +import org.apache.tajo.master.event.QueryEvent; +import org.apache.tajo.master.event.QueryEventType; +import org.apache.tajo.master.event.StageEvent; +import org.apache.tajo.master.event.StageEventType; import org.apache.tajo.plan.LogicalOptimizer; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.LogicalPlanner; import org.apache.tajo.plan.serder.PlanProto; -import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.resource.NodeResources; import org.apache.tajo.session.Session; -import org.apache.tajo.storage.HashShuffleAppenderManager; -import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.history.HistoryReader; -import org.apache.tajo.util.history.HistoryWriter; -import org.apache.tajo.util.metrics.TajoSystemMetrics; import org.apache.tajo.worker.*; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -120,7 +116,7 @@ public final void testKillQueryFromInitState() throws Exception { QueryMaster qm = cluster.getTajoWorkers().get(0).getWorkerContext().getQueryMaster(); QueryMasterTask queryMasterTask = new QueryMasterTask(qm.getContext(), - queryId, session, defaultContext, expr.toJson(), dispatch); + queryId, session, defaultContext, expr.toJson(), NodeResources.createResource(512), dispatch); queryMasterTask.init(conf); queryMasterTask.getQueryTaskContext().getDispatcher().start(); @@ -184,7 +180,7 @@ public final void testIgnoreStageStateFromKilled() throws Exception { QueryMaster qm = cluster.getTajoWorkers().get(0).getWorkerContext().getQueryMaster(); QueryMasterTask queryMasterTask = new QueryMasterTask(qm.getContext(), - queryId, session, defaultContext, expr.toJson(), dispatch); + queryId, session, defaultContext, expr.toJson(), NodeResources.createResource(512), dispatch); queryMasterTask.init(conf); queryMasterTask.getQueryTaskContext().getDispatcher().start(); @@ -220,9 +216,6 @@ public final void testIgnoreStageStateFromKilled() throws Exception { lastStage.getStateMachine().doTransition(StageEventType.SQ_KILL, new StageEvent(lastStage.getId(), StageEventType.SQ_KILL)); - lastStage.getStateMachine().doTransition(StageEventType.SQ_CONTAINER_ALLOCATED, - new StageEvent(lastStage.getId(), StageEventType.SQ_CONTAINER_ALLOCATED)); - lastStage.getStateMachine().doTransition(StageEventType.SQ_SHUFFLE_REPORT, new StageEvent(lastStage.getId(), StageEventType.SQ_SHUFFLE_REPORT)); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 9a6a91d8bd..7fd4dd82a7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -65,7 +65,7 @@ public void setup() { conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); - conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS, 4); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); dispatcher = new AsyncDispatcher(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 450b0aa283..968cf9616a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -24,7 +24,6 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.rm.Worker; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeStatusEvent; import org.junit.After; @@ -159,7 +158,7 @@ public void testResourceReport() throws Exception { statusUpdater.start(); assertEquals(0, statusUpdater.getQueueSize()); - for (int i = 0; i < statusUpdater.getQueueingLimit(); i++) { + for (int i = 0; i < statusUpdater.getQueueingThreshold(); i++) { dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); } barrier.await(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index cbee793693..e43aa2f1b4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -65,7 +65,7 @@ public void setup() { conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); - conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS, 4); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); dispatcher = new AsyncDispatcher(); From b347e04aa5372bfb34f0e283aead42f9ff904112 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 18 Jun 2015 11:16:12 +0900 Subject: [PATCH 05/80] TAJO-1397 --- .../org/apache/tajo/master/QueryManager.java | 18 +++---- .../master/scheduler/SimpleScheduler.java | 4 +- .../querymaster/DefaultTaskScheduler.java | 24 +++++++--- .../tajo/util/history/HistoryWriter.java | 47 +++++++++---------- .../tajo/ws/rs/resources/QueryResource.java | 9 +--- 5 files changed, 53 insertions(+), 49 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java index d2b881f98c..00dd1de672 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java @@ -18,7 +18,6 @@ package org.apache.tajo.master; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.collections.map.LRUMap; @@ -42,7 +41,10 @@ import org.apache.tajo.util.history.HistoryReader; import java.io.IOException; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; /** @@ -114,16 +116,17 @@ public Collection getRunningQueries() { } public synchronized Collection getFinishedQueries() { + Set result = Sets.newTreeSet(); + synchronized (historyCache) { + result.addAll(historyCache.values()); + } + try { - Set result = Sets.newTreeSet(); result.addAll(this.masterContext.getHistoryReader().getQueries(null)); - synchronized (historyCache) { - result.addAll(historyCache.values()); - } return result; } catch (Throwable e) { LOG.error(e, e); - return Lists.newArrayList(); + return result; } } @@ -170,7 +173,6 @@ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, Strin queryInProgress.getQueryInfo().setQueryMaster(""); submittedQueries.put(queryInProgress.getQueryId(), queryInProgress); - //TODO implement scheduler queue QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo("default", queryContext.getUser(), queryInProgress.getQueryId(), 1, queryInProgress.getQueryInfo().getStartTime()); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 4c111c71d2..b57bcf88e8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -110,8 +110,8 @@ private void updateResource() { this.queueInfo.setCurrentCapacity(getResourceCalculator().ratio(clusterResource, maxResource)); } - LOG.info("Scheduler resources \r current: " + getClusterResource() - + "\r maximum: " + getMaximumResourceCapability() + "\r queue: " + queueInfo); + LOG.info("Scheduler resources \n current: " + getClusterResource() + + "\n maximum: " + getMaximumResourceCapability() + "\n queue: " + queueInfo); } @Override diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index aab18bf8fb..e30fc61299 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -950,6 +950,9 @@ public void assignToLeafTasks(LinkedList taskRequests) { LOG.warn("cancel" + proto.getTaskRequest()); } + if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { + stage.getWorkerMap().put(connectionInfo.getId(), addr); + } if(responseProto.getCancellationTaskCount() > 0) { continue; } @@ -1024,12 +1027,6 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); - context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, - null, connectionInfo)); - //taskRequest.getCallback().run(taskAssign.getProto()); - totalAssigned++; - scheduledObjectNum--; - TajoWorkerProtocol.BatchAllocationRequestProto.Builder requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); requestProto.addTaskRequest(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() .setResource(taskRequest.getResponseProto().getResource()) @@ -1060,8 +1057,23 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); + TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); LOG.info(responseProto.getCancellationTaskCount()); + + if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { + stage.getWorkerMap().put(connectionInfo.getId(), addr); + } + + if(responseProto.getCancellationTaskCount() > 0) { + continue; + } + + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, + null, connectionInfo)); + //taskRequest.getCallback().run(taskAssign.getProto()); + totalAssigned++; + scheduledObjectNum--; } catch (Exception e) { LOG.error(e); } diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java index 29fe79a200..25938586ef 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java @@ -32,7 +32,6 @@ import org.apache.tajo.master.QueryInfo; import org.apache.tajo.util.Bytes; import org.apache.tajo.worker.TaskHistory; -import org.apache.tajo.worker.event.NodeStatusEvent; import java.io.Closeable; import java.io.IOException; @@ -229,7 +228,7 @@ public void run() { } try { - if (!histories.isEmpty()) { + if (!stopped.get() && !histories.isEmpty()) { writeHistory(histories); } else { continue; @@ -241,25 +240,23 @@ public void run() { //clean up history file // closing previous writer - synchronized (taskWriters) { - Calendar cal = Calendar.getInstance(); - cal.add(Calendar.HOUR_OF_DAY, -2); - String closeTargetTime = df.format(cal.getTime()); - List closingTargets = new ArrayList(); - - for (String eachWriterTime : taskWriters.keySet()) { - if (eachWriterTime.compareTo(closeTargetTime) <= 0) { - closingTargets.add(eachWriterTime); - } + Calendar cal = Calendar.getInstance(); + cal.add(Calendar.HOUR_OF_DAY, -2); + String closeTargetTime = df.format(cal.getTime()); + List closingTargets = new ArrayList(); + + for (String eachWriterTime : taskWriters.keySet()) { + if (eachWriterTime.compareTo(closeTargetTime) <= 0) { + closingTargets.add(eachWriterTime); } + } - for (String eachWriterTime : closingTargets) { - WriterHolder writerHolder; - writerHolder = taskWriters.remove(eachWriterTime); - if (writerHolder != null) { - LOG.info("Closing task history file: " + writerHolder.path); - IOUtils.cleanup(LOG, writerHolder); - } + for (String eachWriterTime : closingTargets) { + WriterHolder writerHolder; + writerHolder = taskWriters.remove(eachWriterTime); + if (writerHolder != null) { + LOG.info("Closing task history file: " + writerHolder.path); + IOUtils.cleanup(LOG, writerHolder); } } } @@ -425,13 +422,11 @@ private void rollingQuerySummaryWriter() throws Exception { } private void flushTaskHistories() { - synchronized (taskWriters) { - for (WriterHolder holder : taskWriters.values()) { - try { - holder.flush(); - } catch (IOException e) { - LOG.warn(e, e); - } + for (WriterHolder holder : taskWriters.values()) { + try { + holder.flush(); + } catch (IOException e) { + LOG.warn(e, e); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java index a662c4d877..abecc3ad03 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java @@ -155,13 +155,8 @@ public Response run(JerseyResourceDelegateContext context) { for (QueryInProgress queryInProgress: queryManager.getRunningQueries()) { queriesInfo.add(queryInProgress.getQueryInfo()); } - - try { - queriesInfo.addAll(masterContext.getHistoryReader().getQueries(null)); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - return ResourcesUtil.createExceptionResponse(LOG, e.getMessage()); - } + + queriesInfo.addAll(queryManager.getFinishedQueries()); if (state != null) { queriesInfo = selectQueriesInfoByState(queriesInfo, queryState); From b397ff15eedf982de704fb38ec1abe9a0f69df69 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 18 Jun 2015 16:16:28 +0900 Subject: [PATCH 06/80] TAJO-1397 --- tajo-core/pom.xml | 1 - .../tajo/master/TajoMasterClientService.java | 3 - .../tajo/master/container/TajoContainer.java | 173 ------------------ .../master/container/TajoContainerId.java | 171 ----------------- .../container/TajoContainerIdPBImpl.java | 100 ---------- .../master/container/TajoConverterUtils.java | 87 --------- .../tajo/master/event/LocalTaskEvent.java | 1 - .../event/TaskAttemptAssignedEvent.java | 9 +- .../event/TaskAttemptToSchedulerEvent.java | 14 +- .../tajo/master/rm/TajoWorkerContainer.java | 125 ------------- .../tajo/master/rm/TajoWorkerContainerId.java | 94 ---------- .../master/scheduler/SimpleScheduler.java | 70 +++---- .../scheduler/event/SchedulerEventType.java | 9 +- .../querymaster/AbstractTaskScheduler.java | 10 + .../querymaster/DefaultTaskScheduler.java | 117 ++++++------ .../apache/tajo/querymaster/QueryMaster.java | 2 - .../tajo/querymaster/QueryMasterTask.java | 9 +- .../org/apache/tajo/querymaster/Stage.java | 11 +- .../org/apache/tajo/querymaster/Task.java | 24 +-- .../apache/tajo/querymaster/TaskAttempt.java | 9 +- .../java/org/apache/tajo/util/JSPUtil.java | 31 +--- .../tajo/worker/ExecutionBlockContext.java | 8 - .../apache/tajo/worker/ResourceAllocator.java | 29 --- .../java/org/apache/tajo/worker/Task.java | 5 + .../java/org/apache/tajo/worker/TaskImpl.java | 5 + .../apache/tajo/worker/TaskRunnerHistory.java | 152 --------------- .../src/main/proto/ContainerProtocol.proto | 48 ----- .../main/proto/QueryCoordinatorProtocol.proto | 1 - .../src/main/proto/QueryMasterProtocol.proto | 1 - .../main/proto/ResourceTrackerProtocol.proto | 1 - .../src/main/proto/TajoWorkerProtocol.proto | 1 - .../main/resources/webapps/admin/cluster.jsp | 20 +- .../main/resources/webapps/admin/index.jsp | 14 +- .../webapps/admin/query_executor.jsp | 3 +- .../resources/webapps/admin/querytasks.jsp | 11 +- .../src/main/resources/webapps/admin/task.jsp | 1 - .../main/resources/webapps/worker/index.jsp | 26 --- .../resources/webapps/worker/querytasks.jsp | 6 +- .../main/resources/webapps/worker/task.jsp | 3 +- .../webapps/worker/taskcontainers.jsp | 93 ---------- .../resources/webapps/worker/taskdetail.jsp | 28 +-- .../main/resources/webapps/worker/tasks.jsp | 107 ----------- .../apache/tajo/worker/MockTaskExecutor.java | 12 +- .../apache/tajo/worker/TestTaskExecutor.java | 32 ++-- 44 files changed, 194 insertions(+), 1483 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerIdPBImpl.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java delete mode 100644 tajo-core/src/main/proto/ContainerProtocol.proto delete mode 100644 tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp delete mode 100644 tajo-core/src/main/resources/webapps/worker/tasks.jsp diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 4b3936e2f6..6ad6956730 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -164,7 +164,6 @@ --proto_path=../tajo-client/src/main/proto --proto_path=../tajo-plan/src/main/proto --java_out=target/generated-sources/proto - src/main/proto/ContainerProtocol.proto src/main/proto/ResourceTrackerProtocol.proto src/main/proto/QueryMasterProtocol.proto src/main/proto/QueryCoordinatorProtocol.proto diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java index 9c294508bb..449139617e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java @@ -604,9 +604,6 @@ public GetQueryInfoResponse getQueryInfo(RpcController controller, QueryIdReques return builder.build(); } - /** - * It is invoked by TajoContainerProxy. - */ @Override public BoolProto killQuery(RpcController controller, QueryIdRequest request) throws ServiceException { try { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java deleted file mode 100644 index 77562b534a..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java +++ /dev/null @@ -1,173 +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.container; - - -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; -import org.apache.hadoop.yarn.api.records.*; -import org.apache.hadoop.yarn.util.Records; - -/** - * This class is borrowed from the following source code : - * ${hadoop-yarn-api}/src/main/java/org/apache/hadoop/yarn/api/records/Container.java - * - *

TajoContainer represents an allocated resource in the cluster. - *

- * - *

The ResourceManager is the sole authority to allocate any - * TajoContainer to applications. The allocated TajoContainer - * is always on a single node and has a unique {@link org.apache.tajo.master.container.TajoContainerId}. It has - * a specific amount of {@link org.apache.hadoop.yarn.api.records.Resource} allocated.

- * - *

It includes details such as: - *

    - *
  • {@link org.apache.tajo.master.container.TajoContainerId} for the container, which is globally unique.
  • - *
  • - * {@link org.apache.hadoop.yarn.api.records.NodeId} of the node on which it is allocated. - *
  • - *
  • HTTP uri of the node.
  • - *
  • {@link org.apache.hadoop.yarn.api.records.Resource} allocated to the container.
  • - *
  • {@link org.apache.hadoop.yarn.api.records.Priority} at which the container was allocated.
  • - *
  • - * TajoContainer {@link org.apache.hadoop.yarn.api.records.Token} of the container, used to securely verify - * authenticity of the allocation. - *
  • - *
- *

- * - *

Typically, an ApplicationMaster receives the - * TajoContainer from the ResourceManager during - * resource-negotiation and then talks to the NodeManager to - * start/stop containers.

- * - * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) - * @see org.apache.hadoop.yarn.api.ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest) - * @see org.apache.hadoop.yarn.api.ContainerManagementProtocol#stopContainers(org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest) - */ -@Public -@Stable -public abstract class TajoContainer implements Comparable { - - @Private - @Unstable - public static TajoContainer newInstance(TajoContainerId containerId, NodeId nodeId, - String nodeHttpAddress, Resource resource, Priority priority, - Token containerToken) { - TajoContainer container = Records.newRecord(TajoContainer.class); - container.setId(containerId); - container.setNodeId(nodeId); - container.setNodeHttpAddress(nodeHttpAddress); - container.setResource(resource); - container.setPriority(priority); - container.setContainerToken(containerToken); - return container; - } - - /** - * Get the globally unique identifier for the container. - * @return globally unique identifier for the container - */ - @Public - @Stable - public abstract TajoContainerId getId(); - - @Private - @Unstable - public abstract void setId(TajoContainerId id); - - /** - * Get the identifier of the node on which the container is allocated. - * @return identifier of the node on which the container is allocated - */ - @Public - @Stable - public abstract NodeId getNodeId(); - - @Private - @Unstable - public abstract void setNodeId(NodeId nodeId); - - /** - * Get the http uri of the node on which the container is allocated. - * @return http uri of the node on which the container is allocated - */ - @Public - @Stable - public abstract String getNodeHttpAddress(); - - @Private - @Unstable - public abstract void setNodeHttpAddress(String nodeHttpAddress); - - /** - * Get the Resource allocated to the container. - * @return Resource allocated to the container - */ - @Public - @Stable - public abstract Resource getResource(); - - @Private - @Unstable - public abstract void setResource(Resource resource); - - /** - * Get the Priority at which the TajoContainer was - * allocated. - * @return Priority at which the TajoContainer was - * allocated - */ - @Public - @Stable - public abstract Priority getPriority(); - - @Private - @Unstable - public abstract void setPriority(Priority priority); - - /** - * Get the TajoContainerToken for the container. - *

TajoContainerToken is the security token used by the framework - * to verify authenticity of any TajoContainer.

- * - *

The ResourceManager, on container allocation provides a - * secure token which is verified by the NodeManager on - * container launch.

- * - *

Applications do not need to care about TajoContainerToken, they - * are transparently handled by the framework - the allocated - * TajoContainer includes the TajoContainerToken.

- * - * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) - * @see org.apache.hadoop.yarn.api.ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest) - * - * @return TajoContainerToken for the container - */ - @Public - @Stable - public abstract Token getContainerToken(); - - @Private - @Unstable - public abstract void setContainerToken(Token containerToken); -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.java deleted file mode 100644 index 7bc27c662c..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.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.container; - -import java.text.NumberFormat; - -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; - -/** - * This class is borrowed from the following source code : - * ${hadoop-yarn-api}/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java - * - *

TajoContainerId represents a globally unique identifier - * for a {@link org.apache.tajo.master.container.TajoContainer} in the cluster.

- */ -@Public -@Stable -public abstract class TajoContainerId implements Comparable{ - - @Private - @Unstable - public static TajoContainerId newInstance(ApplicationAttemptId appAttemptId, - int containerId) { - TajoContainerId id = new TajoContainerIdPBImpl(); - id.setId(containerId); - id.setApplicationAttemptId(appAttemptId); - id.build(); - return id; - } - - /** - * Get the ApplicationAttemptId of the application to which the - * Container was assigned. - *

- * Note: If containers are kept alive across application attempts via - * {@link org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)} - * the TajoContainerId does not necessarily contain the current - * running application attempt's ApplicationAttemptId This - * container can be allocated by previously exited application attempt and - * managed by the current running attempt thus have the previous application - * attempt's ApplicationAttemptId. - *

- * - * @return ApplicationAttemptId of the application to which the - * Container was assigned - */ - @Public - @Stable - public abstract ApplicationAttemptId getApplicationAttemptId(); - - @Private - @Unstable - protected abstract void setApplicationAttemptId(ApplicationAttemptId atId); - - /** - * Get the identifier of the TajoContainerId. - * @return identifier of the TajoContainerId - */ - @Public - @Stable - public abstract int getId(); - - @Private - @Unstable - protected abstract void setId(int id); - - - // TODO: fail the app submission if attempts are more than 10 or something - private static final ThreadLocal appAttemptIdFormat = - new ThreadLocal() { - @Override - public NumberFormat initialValue() { - NumberFormat fmt = NumberFormat.getInstance(); - fmt.setGroupingUsed(false); - fmt.setMinimumIntegerDigits(2); - return fmt; - } - }; - // TODO: Why thread local? - // ^ NumberFormat instances are not threadsafe - private static final ThreadLocal containerIdFormat = - new ThreadLocal() { - @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 = 435569; - int result = 7507; - result = prime * result + getId(); - result = prime * result + getApplicationAttemptId().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; - TajoContainerId other = (TajoContainerId) obj; - if (!this.getApplicationAttemptId().equals(other.getApplicationAttemptId())) - return false; - if (this.getId() != other.getId()) - return false; - return true; - } - - @Override - public int compareTo(TajoContainerId other) { - if (this.getApplicationAttemptId().compareTo( - other.getApplicationAttemptId()) == 0) { - return this.getId() - other.getId(); - } else { - return this.getApplicationAttemptId().compareTo( - other.getApplicationAttemptId()); - } - - } - - @Override - public String toString() { - NumberFormat fmt = NumberFormat.getInstance(); - fmt.setGroupingUsed(false); - fmt.setMinimumIntegerDigits(4); - - StringBuilder sb = new StringBuilder(); - sb.append("container_"); - ApplicationId appId = getApplicationAttemptId().getApplicationId(); - sb.append(appId.getClusterTimestamp()).append("_"); - sb.append(fmt.format(appId.getId())) - .append("_"); - sb.append( - appAttemptIdFormat.get().format( - getApplicationAttemptId().getAttemptId())).append("_"); - sb.append(containerIdFormat.get().format(getId())); - return sb.toString(); - } - - protected abstract void build(); -} \ No newline at end of file diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerIdPBImpl.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerIdPBImpl.java deleted file mode 100644 index cf9e012add..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerIdPBImpl.java +++ /dev/null @@ -1,100 +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.container; - - -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; -import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; - -import com.google.common.base.Preconditions; -import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.master.container.TajoContainerId; - -/** - * This class is borrowed from the following source code : - * ${hadoop-yarn-common}/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java - * - */ -@Private -@Unstable -public class TajoContainerIdPBImpl extends TajoContainerId { - ContainerProtocol.TajoContainerIdProto proto = null; - ContainerProtocol.TajoContainerIdProto.Builder builder = null; - private ApplicationAttemptId applicationAttemptId = null; - - public TajoContainerIdPBImpl() { - builder = ContainerProtocol.TajoContainerIdProto.newBuilder(); - } - - public TajoContainerIdPBImpl(ContainerProtocol.TajoContainerIdProto proto) { - this.proto = proto; - this.applicationAttemptId = convertFromProtoFormat(proto.getAppAttemptId()); - } - - public ContainerProtocol.TajoContainerIdProto getProto() { - return proto; - } - - @Override - public int getId() { - Preconditions.checkNotNull(proto); - return proto.getId(); - } - - @Override - protected void setId(int id) { - Preconditions.checkNotNull(builder); - builder.setId((id)); - } - - - @Override - public ApplicationAttemptId getApplicationAttemptId() { - return this.applicationAttemptId; - } - - @Override - protected void setApplicationAttemptId(ApplicationAttemptId atId) { - if (atId != null) { - Preconditions.checkNotNull(builder); - builder.setAppAttemptId(convertToProtoFormat(atId)); - } - this.applicationAttemptId = atId; - } - - private ApplicationAttemptIdPBImpl convertFromProtoFormat( - ApplicationAttemptIdProto p) { - return new ApplicationAttemptIdPBImpl(p); - } - - private ApplicationAttemptIdProto convertToProtoFormat( - ApplicationAttemptId t) { - return ((ApplicationAttemptIdPBImpl)t).getProto(); - } - - @Override - protected void build() { - proto = builder.build(); - builder = null; - } -} - diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java deleted file mode 100644 index 88c4823353..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java +++ /dev/null @@ -1,87 +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.container; - - -import static org.apache.hadoop.yarn.util.StringHelper._split; - -import java.net.InetSocketAddress; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.URL; -import org.apache.hadoop.yarn.factories.RecordFactory; -import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; - - -/** - * This class is borrowed from the following source code : - * ${hadoop-yarn-common}/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java - * - * This class contains a set of utilities which help converting data structures - * from/to 'serializableFormat' to/from hadoop/nativejava data structures. - * - */ -@Private -public class TajoConverterUtils { - - public static final String CONTAINER_PREFIX = "container"; - - private static ApplicationAttemptId toApplicationAttemptId( - Iterator it) throws NumberFormatException { - ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()), - Integer.parseInt(it.next())); - ApplicationAttemptId appAttemptId = - ApplicationAttemptId.newInstance(appId, Integer.parseInt(it.next())); - return appAttemptId; - } - - public static String toString(TajoContainerId cId) { - return cId == null ? null : cId.toString(); - } - - public static TajoContainerId toTajoContainerId(String containerIdStr) { - Iterator it = _split(containerIdStr).iterator(); - if (!it.next().equals(CONTAINER_PREFIX)) { - throw new IllegalArgumentException("Invalid TajoContainerId prefix: " - + containerIdStr); - } - try { - ApplicationAttemptId appAttemptID = toApplicationAttemptId(it); - TajoContainerId containerId = - TajoContainerId.newInstance(appAttemptID, Integer.parseInt(it.next())); - return containerId; - } catch (NumberFormatException n) { - throw new IllegalArgumentException("Invalid TajoContainerId: " - + containerIdStr, n); - } - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java index f3cc52aa7d..5a36ba9e3b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java @@ -20,7 +20,6 @@ import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.master.container.TajoContainerId; /** * This event is sent to a running TaskAttempt on a worker. diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java index 16113702a8..08ef80538d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java @@ -20,23 +20,16 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.container.TajoContainerId; public class TaskAttemptAssignedEvent extends TaskAttemptEvent { - private final TajoContainerId cId; private final WorkerConnectionInfo workerConnectionInfo; - public TaskAttemptAssignedEvent(TaskAttemptId id, TajoContainerId cId, + public TaskAttemptAssignedEvent(TaskAttemptId id, WorkerConnectionInfo connectionInfo) { super(id, TaskAttemptEventType.TA_ASSIGNED); - this.cId = cId; this.workerConnectionInfo = connectionInfo; } - public TajoContainerId getContainerId() { - return cId; - } - public WorkerConnectionInfo getWorkerConnectionInfo(){ return workerConnectionInfo; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java index 5a016fb0ee..6b939732da 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java @@ -22,7 +22,6 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.querymaster.TaskAttempt; -import org.apache.tajo.master.container.TajoContainerId; public class TaskAttemptToSchedulerEvent extends TaskSchedulerEvent { private final TaskAttemptScheduleContext context; @@ -44,7 +43,6 @@ public TaskAttemptScheduleContext getContext() { } public static class TaskAttemptScheduleContext { - private TajoContainerId containerId; private String host; private RpcCallback callback; @@ -52,22 +50,12 @@ public TaskAttemptScheduleContext() { } - public TaskAttemptScheduleContext(TajoContainerId containerId, - String host, + public TaskAttemptScheduleContext(String host, RpcCallback callback) { - this.containerId = containerId; this.host = host; this.callback = callback; } - public TajoContainerId getContainerId() { - return containerId; - } - - public void setContainerId(TajoContainerId containerId) { - this.containerId = containerId; - } - public String getHost() { return host; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java deleted file mode 100644 index 8c5b96c5c5..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java +++ /dev/null @@ -1,125 +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 org.apache.hadoop.yarn.api.records.*; -import org.apache.tajo.master.container.TajoContainer; -import org.apache.tajo.master.container.TajoContainerId; - - -public class TajoWorkerContainer extends TajoContainer { - TajoContainerId id; - NodeId nodeId; - Worker worker; - - public Worker getWorkerResource() { - return worker; - } - - public void setWorkerResource(Worker workerResource) { - this.worker = workerResource; - } - - @Override - public TajoContainerId getId() { - return id; - } - - @Override - public void setId(TajoContainerId 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 Token getContainerToken() { - return null; //To change body of implemented methods use File | Settings | File Templates. - } - - @Override - public void setContainerToken(Token containerToken) { - //To change body of implemented methods use File | Settings | File Templates. - } - - @Override - public int compareTo(TajoContainer container) { - return getId().compareTo(container.getId()); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - TajoWorkerContainer that = (TajoWorkerContainer) o; - - if (id != null ? !id.equals(that.id) : that.id != null) return false; - if (nodeId != null ? !nodeId.equals(that.nodeId) : that.nodeId != null) return false; - if (worker != null ? !worker.equals(that.worker) : that.worker != null) return false; - - return true; - } - - @Override - public int hashCode() { - int result = id != null ? id.hashCode() : 0; - result = 31 * result + (nodeId != null ? nodeId.hashCode() : 0); - result = 31 * result + (worker != null ? worker.hashCode() : 0); - return result; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java deleted file mode 100644 index 184de71fdd..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java +++ /dev/null @@ -1,94 +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 org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.proto.YarnProtos; -import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.master.container.TajoContainerId; - -public class TajoWorkerContainerId extends TajoContainerId { - 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; - } - - public ContainerProtocol.TajoContainerIdProto getProto() { - YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder() - .setClusterTimestamp(applicationAttemptId.getApplicationId().getClusterTimestamp()) - .setId(applicationAttemptId.getApplicationId().getId()) - .build(); - - YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder() - .setAttemptId(applicationAttemptId.getAttemptId()) - .setApplicationId(appIdProto) - .build(); - - return ContainerProtocol.TajoContainerIdProto.newBuilder() - .setAppAttemptId(attemptIdProto) - .setAppId(appIdProto) - .setId(id) - .build(); - } - - public static ContainerProtocol.TajoContainerIdProto getContainerIdProto(TajoContainerId containerId) { - if(containerId instanceof TajoWorkerContainerId) { - return ((TajoWorkerContainerId)containerId).getProto(); - } else { - YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder() - .setClusterTimestamp(containerId.getApplicationAttemptId().getApplicationId().getClusterTimestamp()) - .setId(containerId.getApplicationAttemptId().getApplicationId().getId()) - .build(); - - YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder() - .setAttemptId(containerId.getApplicationAttemptId().getAttemptId()) - .setApplicationId(appIdProto) - .build(); - - return ContainerProtocol.TajoContainerIdProto.newBuilder() - .setAppAttemptId(attemptIdProto) - .setAppId(appIdProto) - .setId(containerId.getId()) - .build(); - } - } - - @Override - protected void build() { - - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index b57bcf88e8..ceb2414bf7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -51,18 +51,17 @@ public class SimpleScheduler extends AbstractQueryScheduler { private static final Log LOG = LogFactory.getLog(SimpleScheduler.class); private static final String DEFAULT_QUEUE_NAME = "default"; - + private static final Comparator COMPARATOR = new SchedulingAlgorithms.FifoComparator(); private volatile boolean isStopped = false; private final TajoMaster.MasterContext masterContext; - private final TajoRMContext rmContext; + private final TajoRMContext rmContext; private final BlockingQueue queryQueue; private final Map pendingQueryMap = Maps.newHashMap(); - private final Map assignedQueryMasterMap = Maps.newHashMap(); + private final Map assignedQueryMasterMap = Maps.newHashMap(); private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator(); - private static Comparator COMPARATOR = new SchedulingAlgorithms.FifoComparator(); private final Thread queryProcessor; @@ -178,26 +177,45 @@ public int getNumClusterNodes() { public List reserve(QueryId queryId, QueryCoordinatorProtocol.NodeResourceRequestProto request) { if (LOG.isDebugEnabled()) { - LOG.debug("Request:" + request.toString() + "Cluster resource: " + getClusterResource()); + LOG.debug("Request:" + request.toString() + "Cluster resource: " + getClusterResource()); } - List reservedResources = Lists.newArrayList(); + List reservedResources; NodeResource capacity = new NodeResource(request.getCapacity()); if (!NodeResources.fitsIn(capacity, getClusterResource())) { - return reservedResources; + return Lists.newArrayList(); } - AllocationResourceProto.Builder resourceBuilder = AllocationResourceProto.newBuilder(); LinkedList workers = new LinkedList(); if (request.getCandidateNodesCount() > 0) { workers.addAll(request.getCandidateNodesList()); } - int allocatedResources = 0; int requiredContainers = request.getNumContainers(); + // reserve resource to the candidate workers for locality + reservedResources = reserveClusterResource(workers, capacity, requiredContainers); + + // reserve resource in random workers + if (reservedResources.size() < requiredContainers) { + LinkedList randomWorkers = new LinkedList(getRMContext().getWorkers().keySet()); + randomWorkers.removeAll(workers); + Collections.shuffle(randomWorkers); + + reservedResources.addAll(reserveClusterResource( + randomWorkers, capacity, requiredContainers - reservedResources.size())); + } + + return reservedResources; + } + + private List reserveClusterResource(List workers, + NodeResource capacity, int requiredNum) { + + List reservedResources = Lists.newArrayList(); + AllocationResourceProto.Builder resourceBuilder = AllocationResourceProto.newBuilder(); + int allocatedResources = 0; - // reserve resource in candidate workers while (workers.size() > 0) { Iterator iter = workers.iterator(); while (iter.hasNext()) { @@ -222,41 +240,11 @@ public int getNumClusterNodes() { } } - if (allocatedResources >= requiredContainers) { + if (allocatedResources >= requiredNum) { return reservedResources; } } } - - // reserve resource in random workers - if(allocatedResources == 0 || allocatedResources < requiredContainers) { - LinkedList randomWorkers = new LinkedList(getRMContext().getWorkers().keySet()); - randomWorkers.removeAll(workers); - Collections.shuffle(randomWorkers); - - for (int workerId : randomWorkers) { - - Worker worker = getRMContext().getWorkers().get(workerId); - if (worker == null) { - LOG.warn("Can't found the worker :" + workerId); - continue; - } else { - if (NodeResources.fitsIn(capacity, worker.getAvailableResource())) { - NodeResources.subtractFrom(getClusterResource(), capacity); - NodeResources.subtractFrom(worker.getAvailableResource(), capacity); - allocatedResources++; - resourceBuilder.setResource(capacity.getProto()); - resourceBuilder.setWorkerId(workerId); - reservedResources.add(resourceBuilder.build()); - } - } - - if (allocatedResources >= requiredContainers) { - break; - } - } - } - return reservedResources; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEventType.java index 725cbcd12a..93fa032b75 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEventType.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/SchedulerEventType.java @@ -20,12 +20,7 @@ public enum SchedulerEventType { - // Source: QM + // consumer: Scheduler RESOURCE_RESERVE, - - RESOURCE_UPDATE, - - // Source: MASTER - QM_SUBMIT, - QM_STOP + RESOURCE_UPDATE } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java index e45f27401e..eb8c93da3d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java @@ -18,17 +18,21 @@ package org.apache.tajo.querymaster; +import com.google.common.collect.Sets; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.master.event.TaskRequestEvent; import org.apache.tajo.master.event.TaskSchedulerEvent; +import java.util.Set; + public abstract class AbstractTaskScheduler extends AbstractService implements EventHandler { protected int hostLocalAssigned; protected int rackLocalAssigned; protected int totalAssigned; + protected Set leafTaskHosts = Sets.newHashSet(); /** * Construct the service. @@ -52,5 +56,11 @@ public int getTotalAssigned() { } public abstract void handleTaskRequestEvent(TaskRequestEvent event); + public abstract void releseTaskAttempt(TaskAttempt taskAttempt); public abstract int remainingScheduledObjectNum(); + + + public Set getLeafTaskHosts(){ + return leafTaskHosts; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index e30fc61299..4a94d6a3d9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -37,11 +37,9 @@ import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.container.TajoContainerId; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; import org.apache.tajo.master.event.TaskSchedulerEvent.EventType; -import org.apache.tajo.master.rm.TajoWorkerContainerId; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.serder.LogicalNodeSerializer; import org.apache.tajo.plan.serder.PlanProto; @@ -54,7 +52,6 @@ import org.apache.tajo.storage.DataLocation; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.storage.fragment.Fragment; -import org.apache.tajo.util.ApplicationIdUtils; import org.apache.tajo.util.NetUtils; import org.apache.tajo.worker.FetchImpl; @@ -150,13 +147,6 @@ public void stop() { } } - // Return all of request callbacks instantly. - if(taskRequests != null){ - for (TaskRequestEvent req : taskRequests.taskRequestQueue) { -// req.getCallback().run(stopTaskRunnerReq); - } - } - LOG.info("Task Scheduler stopped"); super.stop(); } @@ -260,6 +250,18 @@ public void handle(TaskSchedulerEvent event) { } } + private List getWorkerIds(Collection hosts){ + List workerIds = Lists.newArrayList(); + if(hosts.isEmpty()) return workerIds; + + for (WorkerConnectionInfo worker : stage.getContext().getWorkerMap().values()) { + if(hosts.contains(worker.getHost())){ + workerIds.add(worker.getId()); + } + } + return workerIds; + } + @Override public void handleTaskRequestEvent(TaskRequestEvent event) { @@ -275,12 +277,13 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); request.setCapacity(NodeResources.createResource(512).getProto()); request.setNumContainers(scheduledObjectNum); - request.setPriority(1); + request.setPriority(stage.getPriority()); request.setQueryId(context.getMasterContext().getQueryId().getProto()); request.setQueue("default"); request.setType(QueryCoordinatorProtocol.ResourceType.LEAF); request.setUserId("test"); - request.setRunningTasks(0); + request.setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()); + request.addAllCandidateNodes(getWorkerIds(getLeafTaskHosts())); masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); QueryCoordinatorProtocol.NodeResourceResponseProto responseProto = callBack.get(); @@ -357,6 +360,17 @@ public int size() { } } + public void releseTaskAttempt(TaskAttempt taskAttempt) { + if (taskAttempt.isLeafTask() && taskAttempt.getWorkerConnectionInfo() != null) { + + HostVolumeMapping mapping = + scheduledRequests.leafTaskHostMapping.get(taskAttempt.getWorkerConnectionInfo().getHost()); + if (mapping != null) { + int volumeId = mapping.lastAssignedVolumeId.remove(taskAttempt.getId()); + mapping.decreaseConcurrency(volumeId); + } + } + } /** * One worker can have multiple running task runners. HostVolumeMapping * describes various information for one worker, including : @@ -391,8 +405,7 @@ public class HostVolumeMapping { private Map> unassignedTaskForEachVolume = Collections.synchronizedMap(new HashMap>()); /** A value is last assigned volume id for each task runner */ - private HashMap lastAssignedVolumeId = new HashMap(); + private HashMap lastAssignedVolumeId = Maps.newHashMap(); /** * A key is disk volume id, and a value is the load of this volume. * This load is measured by counting how many number of tasks are running. @@ -432,24 +445,18 @@ public synchronized void addTaskAttempt(int volumeId, TaskAttempt attemptId){ * 2. unknown block or Non-splittable task in host * 3. remote tasks. unassignedTaskForEachVolume is only contained local task. so it will be null */ - public synchronized TaskAttemptId getLocalTask(TajoContainerId containerId) { - int volumeId; + public synchronized TaskAttemptId getLocalTask() { + int volumeId = getLowestVolumeId(); TaskAttemptId taskAttemptId = null; - if (!lastAssignedVolumeId.containsKey(containerId)) { - volumeId = getLowestVolumeId(); - increaseConcurrency(containerId, volumeId); - } else { - volumeId = lastAssignedVolumeId.get(containerId); - } - + increaseConcurrency(volumeId); if (unassignedTaskForEachVolume.size() > 0) { int retry = unassignedTaskForEachVolume.size(); do { //clean and get a remaining local task taskAttemptId = getAndRemove(volumeId); if(!unassignedTaskForEachVolume.containsKey(volumeId)) { - decreaseConcurrency(containerId); + decreaseConcurrency(volumeId); if (volumeId > REMOTE) { diskVolumeLoads.remove(volumeId); } @@ -458,7 +465,7 @@ public synchronized TaskAttemptId getLocalTask(TajoContainerId containerId) { if (taskAttemptId == null) { //reassign next volume volumeId = getLowestVolumeId(); - increaseConcurrency(containerId, volumeId); + increaseConcurrency(volumeId); retry--; } else { break; @@ -467,6 +474,8 @@ public synchronized TaskAttemptId getLocalTask(TajoContainerId containerId) { } else { this.remainTasksNum.set(0); } + + lastAssignedVolumeId.put(taskAttemptId, volumeId); return taskAttemptId; } @@ -539,11 +548,10 @@ private synchronized void removeTaskAttempt(int volumeId, TaskAttempt taskAttemp /** * Increase the count of running tasks and disk loads for a certain task runner. * - * @param containerId The task runner identifier * @param volumeId Volume identifier * @return the volume load (i.e., how many running tasks use this volume) */ - private synchronized int increaseConcurrency(TajoContainerId containerId, int volumeId) { + private synchronized int increaseConcurrency(int volumeId) { int concurrency = 1; if (diskVolumeLoads.containsKey(volumeId)) { @@ -561,16 +569,14 @@ private synchronized int increaseConcurrency(TajoContainerId containerId, int vo + ", Remote Concurrency : " + concurrency); } diskVolumeLoads.put(volumeId, concurrency); - lastAssignedVolumeId.put(containerId, volumeId); return concurrency; } /** * Decrease the count of running tasks of a certain task runner */ - private synchronized void decreaseConcurrency(TajoContainerId containerId){ - Integer volumeId = lastAssignedVolumeId.get(containerId); - if(volumeId != null && diskVolumeLoads.containsKey(volumeId)){ + private synchronized void decreaseConcurrency(int volumeId){ + if(diskVolumeLoads.containsKey(volumeId)){ Integer concurrency = diskVolumeLoads.get(volumeId); if(concurrency > 0){ diskVolumeLoads.put(volumeId, concurrency - 1); @@ -580,7 +586,6 @@ private synchronized void decreaseConcurrency(TajoContainerId containerId){ } } } - lastAssignedVolumeId.remove(containerId); } /** @@ -606,12 +611,8 @@ public int getLowestVolumeId(){ } } - public boolean isAssigned(TajoContainerId containerId){ - return lastAssignedVolumeId.containsKey(containerId); - } - - public boolean isRemote(TajoContainerId containerId){ - Integer volumeId = lastAssignedVolumeId.get(containerId); + public boolean isRemote(TaskAttemptId taskAttemptId){ + Integer volumeId = lastAssignedVolumeId.get(taskAttemptId); if(volumeId == null || volumeId > REMOTE){ return false; } else { @@ -641,6 +642,10 @@ public String getHost() { public String getRack() { return rack; } + + public int getAssignedVolumeId(TaskAttemptId attemptId) { + return lastAssignedVolumeId.get(attemptId); + } } public void cancel(TaskAttempt taskAttempt) { @@ -669,6 +674,7 @@ private synchronized void addLeafTask(TaskAttemptToSchedulerEvent event) { for (DataLocation location : locations) { String host = location.getHost(); + leafTaskHosts.add(host); HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); if (hostVolumeMapping == null) { @@ -712,12 +718,12 @@ public int nonLeafTaskNum() { public Set assignedRequest = new HashSet(); - private TaskAttemptId allocateLocalTask(String host, TajoContainerId containerId){ + private TaskAttemptId allocateLocalTask(String host){ HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); if (hostVolumeMapping != null) { //tajo host is located in hadoop datanode for (int i = 0; i < hostVolumeMapping.getRemainingLocalTaskSize(); i++) { - TaskAttemptId attemptId = hostVolumeMapping.getLocalTask(containerId); + TaskAttemptId attemptId = hostVolumeMapping.getLocalTask(); if(attemptId == null) break; //find remaining local task @@ -840,36 +846,33 @@ public void assignToLeafTasks(LinkedList taskRequests) { } } - TajoWorkerContainerId containerId = new TajoWorkerContainerId(); - - containerId.setApplicationAttemptId( - ApplicationIdUtils.createApplicationAttemptId(taskRequest.getExecutionBlockId().getQueryId())); - containerId.setId(0); - - - LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," + - "containerId=" + containerId); + if (LOG.isDebugEnabled()) { + LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," + + "worker=" + connectionInfo.getHostAndPeerRpcPort()); + } ////////////////////////////////////////////////////////////////////// // disk or host-local allocation ////////////////////////////////////////////////////////////////////// - TaskAttemptId attemptId = allocateLocalTask(host, containerId); + TaskAttemptId attemptId = allocateLocalTask(host); if (attemptId == null) { // if a local task cannot be found HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); if(hostVolumeMapping != null) { - if(!hostVolumeMapping.isRemote(containerId)){ + if(!hostVolumeMapping.isRemote(attemptId)){ // assign to remote volume - hostVolumeMapping.decreaseConcurrency(containerId); - hostVolumeMapping.increaseConcurrency(containerId, HostVolumeMapping.REMOTE); + int volumeId = hostVolumeMapping.getAssignedVolumeId(attemptId); + hostVolumeMapping.decreaseConcurrency(volumeId); + hostVolumeMapping.increaseConcurrency(HostVolumeMapping.REMOTE); } // this part is remote concurrency management of a tail tasks int tailLimit = Math.max(remainingScheduledObjectNum() / (leafTaskHostMapping.size() * 2), 1); if(hostVolumeMapping.getRemoteConcurrency() > tailLimit){ //release container - hostVolumeMapping.decreaseConcurrency(containerId); + int volumeId = hostVolumeMapping.getAssignedVolumeId(attemptId); + hostVolumeMapping.decreaseConcurrency(volumeId); //taskRequest.getCallback().run(stopTaskRunnerReq); continue; } @@ -960,8 +963,7 @@ public void assignToLeafTasks(LinkedList taskRequests) { LOG.error(e); } - context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, - containerId, connectionInfo)); + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); assignedRequest.add(attemptId); scheduledObjectNum--; @@ -1069,8 +1071,7 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { continue; } - context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, - null, connectionInfo)); + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); //taskRequest.getCallback().run(taskAssign.getProto()); totalAssigned++; scheduledObjectNum--; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index c1ebd802b8..a78bbe8d56 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -297,10 +297,8 @@ public void stopQuery(QueryId queryId) { QueryHistory queryHisory = query.getQueryHistory(); if (queryHisory != null) { try { - long startTime = System.currentTimeMillis(); query.context.getQueryMasterContext().getWorkerContext(). getTaskHistoryWriter().appendAndFlush(queryHisory); - LOG.info("QueryHistory write delay:" + (System.currentTimeMillis() - startTime)); } catch (Throwable e) { LOG.warn(e, e); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index c592c133be..6fc80eb6c3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -68,6 +68,7 @@ import org.apache.tajo.util.metrics.TajoMetrics; import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter; import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; +import org.apache.tajo.worker.event.NodeStatusEvent; import java.io.IOException; import java.net.InetSocketAddress; @@ -192,8 +193,12 @@ public void serviceStop() throws Exception { LOG.info("Stopping QueryMasterTask:" + queryId); //release QM resource - getQueryTaskContext().getQueryMasterContext().getWorkerContext(). - getNodeResourceManager().getDispatcher().getEventHandler().handle(new NodeResourceDeallocateEvent(allocation)); + getQueryTaskContext().getQueryMasterContext().getWorkerContext().getNodeResourceManager().getDispatcher() + .getEventHandler().handle(new NodeResourceDeallocateEvent(allocation)); + + //flush current node resource + getQueryTaskContext().getQueryMasterContext().getWorkerContext().getNodeResourceManager().getDispatcher() + .getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); if (!queryContext.getBool(SessionVars.DEBUG_ENABLED)) { cleanupQuery(getQueryId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 520383608a..858b30d0db 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -274,11 +274,11 @@ StageEventType.SQ_KILL, new KillTasksTransition()) private final Lock readLock; private final Lock writeLock; - private int totalScheduledObjectsCount; - private int completedTaskCount = 0; - private int succeededObjectCount = 0; - private int killedObjectCount = 0; - private int failedObjectCount = 0; + private volatile int totalScheduledObjectsCount; + private volatile int completedTaskCount = 0; + private volatile int succeededObjectCount = 0; + private volatile int killedObjectCount = 0; + private volatile int failedObjectCount = 0; private TaskSchedulerContext schedulerContext; private List hashShuffleIntermediateEntries = Lists.newArrayList(); private AtomicInteger completedShuffleTasks = new AtomicInteger(0); @@ -1165,6 +1165,7 @@ public void transition(Stage stage, stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_FAILED)); } else { stage.completedTaskCount++; + stage.getTaskScheduler().releseTaskAttempt(task.getLastAttempt()); //FIXME if (taskEvent.getState() == TaskState.SUCCEEDED) { stage.succeededObjectCount++; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java index d2be97329c..9a9c3ac9fb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java @@ -37,6 +37,7 @@ import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; import org.apache.tajo.master.TaskState; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; import org.apache.tajo.plan.logical.*; @@ -45,7 +46,6 @@ import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.storage.fragment.FragmentConvertor; import org.apache.tajo.util.NumberUtil; -import org.apache.tajo.util.Pair; import org.apache.tajo.util.TajoIdUtils; import org.apache.tajo.util.history.TaskHistory; import org.apache.tajo.worker.FetchImpl; @@ -87,9 +87,8 @@ public class Task implements EventHandler { private TaskAttemptId lastAttemptId; private TaskAttemptId successfulAttempt; - private String succeededHost; - private int succeededHostPort; - private int succeededPullServerPort; + + private WorkerConnectionInfo succeededWorker; private int failedAttempts; private int finishedAttempts; // finish are total of success, failed and killed @@ -253,7 +252,11 @@ private TaskHistory makeTaskHistory() { taskHistory.setState(lastAttempt.getState().toString()); taskHistory.setProgress(lastAttempt.getProgress()); } - taskHistory.setHostAndPort(succeededHost + ":" + succeededHostPort); + + if(getSucceededWorker() != null) { + taskHistory.setHostAndPort(succeededWorker.getHostAndPeerRpcPort()); + } + taskHistory.setRetryCount(this.getRetryCount()); taskHistory.setLaunchTime(launchTime); taskHistory.setFinishTime(finishTime); @@ -358,8 +361,8 @@ public List getDataLocations() { return dataLocations; } - public String getSucceededHost() { - return succeededHost; + public WorkerConnectionInfo getSucceededWorker() { + return succeededWorker; } public void addFetches(String tableId, Collection fetches) { @@ -609,9 +612,7 @@ public void transition(Task task, TaskAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId()); task.successfulAttempt = attemptEvent.getTaskAttemptId(); - task.succeededHost = attempt.getWorkerConnectionInfo().getHost(); - task.succeededHostPort = attempt.getWorkerConnectionInfo().getPeerRpcPort(); - task.succeededPullServerPort = attempt.getWorkerConnectionInfo().getPullServerPort(); + task.succeededWorker = attempt.getWorkerConnectionInfo(); task.finishTask(); task.eventHandler.handle(new StageTaskEvent(event.getTaskId(), TaskState.SUCCEEDED)); @@ -628,8 +629,7 @@ public void transition(Task task, TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event; TaskAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId()); task.launchTime = System.currentTimeMillis(); - task.succeededHost = attempt.getWorkerConnectionInfo().getHost(); - task.succeededHostPort = attempt.getWorkerConnectionInfo().getPeerRpcPort(); + task.succeededWorker = attempt.getWorkerConnectionInfo(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java index cfce0c9ac0..afd3f2a3dc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java @@ -23,8 +23,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.*; -import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TajoProtos.TaskAttemptState; +import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport; @@ -34,7 +34,6 @@ import org.apache.tajo.master.event.TaskSchedulerEvent.EventType; import org.apache.tajo.querymaster.Task.IntermediateEntry; import org.apache.tajo.querymaster.Task.PullHost; -import org.apache.tajo.master.container.TajoContainerId; import java.util.ArrayList; import java.util.EnumSet; @@ -55,7 +54,6 @@ public class TaskAttempt implements EventHandler { private final Task task; final EventHandler eventHandler; - private TajoContainerId containerId; private WorkerConnectionInfo workerConnectionInfo; private int expire; @@ -214,10 +212,6 @@ public WorkerConnectionInfo getWorkerConnectionInfo() { return this.workerConnectionInfo; } - public void setContainerId(TajoContainerId containerId) { - this.containerId = containerId; - } - public synchronized void setExpireTime(int expire) { this.expire = expire; } @@ -311,7 +305,6 @@ public void transition(TaskAttempt taskAttempt, throw new IllegalArgumentException("event should be a TaskAttemptAssignedEvent type."); } TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event; - taskAttempt.containerId = castEvent.getContainerId(); taskAttempt.workerConnectionInfo = castEvent.getWorkerConnectionInfo(); taskAttempt.eventHandler.handle( new TaskTAttemptEvent(taskAttempt.getId(), diff --git a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java index 5021d83cf8..7641320ba5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java @@ -30,7 +30,6 @@ import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.util.history.StageHistory; import org.apache.tajo.util.history.TaskHistory; -import org.apache.tajo.worker.TaskRunnerHistory; import java.text.DecimalFormat; import java.util.*; @@ -64,28 +63,6 @@ public static void sortTaskHistory(List tasks, String sortField, St Collections.sort(tasks, new TaskHistoryComparator(sortField, "asc".equals(sortOrder))); } -// public static void sortTaskRunner(List taskRunners) { -// Collections.sort(taskRunners, new Comparator() { -// @Override -// public int compare(TaskRunner taskRunner, TaskRunner taskRunner2) { -// return taskRunner.getId().compareTo(taskRunner2.getId()); -// } -// }); -// } - - public static void sortTaskRunnerHistory(List histories) { - Collections.sort(histories, new Comparator() { - @Override - public int compare(TaskRunnerHistory h1, TaskRunnerHistory h2) { - int value = h1.getExecutionBlockId().compareTo(h2.getExecutionBlockId()); - if(value == 0){ - return h1.getContainerId().compareTo(h2.getContainerId()); - } - return value; - } - }); - } - public static String getElapsedTime(long startTime, long finishTime) { if(startTime == 0) { return "-"; @@ -219,8 +196,8 @@ public int compare(Task task, Task task2) { if("id".equals(sortField)) { return task.getId().compareTo(task2.getId()); } else if("host".equals(sortField)) { - String host1 = task.getSucceededHost() == null ? "-" : task.getSucceededHost(); - String host2 = task2.getSucceededHost() == null ? "-" : task2.getSucceededHost(); + String host1 = task.getSucceededWorker() == null ? "-" : task.getSucceededWorker().getHost(); + String host2 = task2.getSucceededWorker() == null ? "-" : task2.getSucceededWorker().getHost(); return host1.compareTo(host2); } else if("runTime".equals(sortField)) { return compareLong(task.getRunningTime(), task2.getRunningTime()); @@ -233,8 +210,8 @@ public int compare(Task task, Task task2) { if("id".equals(sortField)) { return task2.getId().compareTo(task.getId()); } else if("host".equals(sortField)) { - String host1 = task.getSucceededHost() == null ? "-" : task.getSucceededHost(); - String host2 = task2.getSucceededHost() == null ? "-" : task2.getSucceededHost(); + String host1 = task.getSucceededWorker() == null ? "-" : task.getSucceededWorker().getHost(); + String host2 = task2.getSucceededWorker() == null ? "-" : task2.getSucceededWorker().getHost(); return host2.compareTo(host1); } else if("runTime".equals(sortField)) { if(task2.getLaunchTime() == 0) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 9596d3ceff..a850ad3589 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -95,9 +95,6 @@ public class ExecutionBlockContext { // It keeps all of the query unit attempts while a TaskRunner is running. private final ConcurrentMap tasks = Maps.newConcurrentMap(); - @Deprecated - private final ConcurrentMap histories = Maps.newConcurrentMap(); - private final Map taskHistories = Maps.newTreeMap(); public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, StartExecutionBlockRequestProto request) @@ -273,11 +270,6 @@ public void fatalError(TaskAttemptId taskAttemptId, String message) { getStub().fatalError(null, builder.build(), NullCallback.get()); } -// public TaskRunnerHistory createTaskRunnerHistory(TaskRunner runner){ -// histories.putIfAbsent(runner.getId(), new TaskRunnerHistory(runner.getContainerId(), executionBlockId)); -// return histories.get(runner.getId()); -// } - public TajoWorker.WorkerContext getWorkerContext(){ return workerContext; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java deleted file mode 100644 index b713e70904..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java +++ /dev/null @@ -1,29 +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.worker; - -import org.apache.tajo.ipc.ContainerProtocol; -import org.apache.tajo.master.container.TajoContainerId; - -public interface ResourceAllocator { - public void allocateTaskWorker(); - public TajoContainerId makeContainerId(ContainerProtocol.TajoContainerIdProto containerId); - public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext, - int numTasks, int memoryMBPerTask); -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index c84994003e..32d3f0cae0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -21,6 +21,7 @@ import org.apache.tajo.ipc.TajoWorkerProtocol; import java.io.IOException; +import java.util.List; public interface Task { @@ -49,4 +50,8 @@ public interface Task { ExecutionBlockContext getExecutionBlockContext(); TajoWorkerProtocol.TaskStatusProto getReport(); + + TaskHistory createTaskHistory(); + + List getFetchers(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java index be3960b0ca..88387a7008 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -485,6 +485,7 @@ public void cleanup() { stopScriptExecutors(); } + @Override public TaskHistory createTaskHistory() { TaskHistory taskHistory = null; try { @@ -529,6 +530,10 @@ public TaskHistory createTaskHistory() { return taskHistory; } + public List getFetchers() { + return fetcherRunners; + } + public int hashCode() { return context.hashCode(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java deleted file mode 100644 index 16d32d443d..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java +++ /dev/null @@ -1,152 +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.worker; - -import com.google.common.base.Objects; -import com.google.common.collect.Maps; -import org.apache.hadoop.service.Service; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.common.ProtoObject; -import org.apache.tajo.master.container.TajoContainerId; -import org.apache.tajo.master.container.TajoConverterUtils; - -import java.util.Collections; -import java.util.Map; - -import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskHistoryProto; -import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskRunnerHistoryProto; - -/** - * The history class for TaskRunner processing. - */ -@Deprecated -public class TaskRunnerHistory implements ProtoObject { - - private Service.STATE state; - private TajoContainerId containerId; - private long startTime; - private long finishTime; - private ExecutionBlockId executionBlockId; - private Map taskHistoryMap = null; - - public TaskRunnerHistory(TajoContainerId containerId, ExecutionBlockId executionBlockId) { - init(); - this.containerId = containerId; - this.executionBlockId = executionBlockId; - } - - public TaskRunnerHistory(TaskRunnerHistoryProto proto) { - this.state = Service.STATE.valueOf(proto.getState()); - this.containerId = TajoConverterUtils.toTajoContainerId(proto.getContainerId()); - this.startTime = proto.getStartTime(); - this.finishTime = proto.getFinishTime(); - this.executionBlockId = new ExecutionBlockId(proto.getExecutionBlockId()); - this.taskHistoryMap = Maps.newTreeMap(); - for (TaskHistoryProto taskHistoryProto : proto.getTaskHistoriesList()) { - TaskHistory taskHistory = new TaskHistory(taskHistoryProto); - taskHistoryMap.put(taskHistory.getTaskAttemptId(), taskHistory); - } - } - - private void init() { - this.taskHistoryMap = Maps.newHashMap(); - } - - public int size() { - return this.taskHistoryMap.size(); - } - - @Override - public int hashCode() { - return Objects.hashCode(containerId, executionBlockId, state, startTime, - finishTime, taskHistoryMap.size()); - } - - @Override - public boolean equals(Object o) { - if (o instanceof TaskRunnerHistory) { - TaskRunnerHistory other = (TaskRunnerHistory) o; - return getProto().equals(other.getProto()); - } - return false; - } - - @Override - public TaskRunnerHistoryProto getProto() { - TaskRunnerHistoryProto.Builder builder = TaskRunnerHistoryProto.newBuilder(); - builder.setContainerId(containerId.toString()); - builder.setState(state.toString()); - builder.setExecutionBlockId(executionBlockId.getProto()); - builder.setStartTime(startTime); - builder.setFinishTime(finishTime); - for (TaskHistory taskHistory : taskHistoryMap.values()){ - builder.addTaskHistories(taskHistory.getProto()); - } - return builder.build(); - } - - 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 ExecutionBlockId getExecutionBlockId() { - return executionBlockId; - } - - public Service.STATE getState() { - return state; - } - - public void setState(Service.STATE state) { - this.state = state; - } - - public TajoContainerId getContainerId() { - return containerId; - } - - public void setContainerId(TajoContainerId containerId) { - this.containerId = containerId; - } - - public TaskHistory getTaskHistory(TaskAttemptId taskAttemptId) { - return taskHistoryMap.get(taskAttemptId); - } - - public Map getTaskHistoryMap() { - return Collections.unmodifiableMap(taskHistoryMap); - } - - public void addTaskHistory(TaskAttemptId taskAttemptId, TaskHistory taskHistory) { - taskHistoryMap.put(taskAttemptId, taskHistory); - } -} diff --git a/tajo-core/src/main/proto/ContainerProtocol.proto b/tajo-core/src/main/proto/ContainerProtocol.proto deleted file mode 100644 index df7a450b5f..0000000000 --- a/tajo-core/src/main/proto/ContainerProtocol.proto +++ /dev/null @@ -1,48 +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. - */ - -/** - * These .proto interfaces are public and stable. - * Please see http://wiki.apache.org/hadoop/Compatibility - * for what changes are allowed for a *stable* .proto interface. - */ - -option java_package = "org.apache.tajo.ipc"; -option java_outer_classname = "ContainerProtocol"; -option java_generic_services = true; -option java_generate_equals_and_hash = true; - -package hadoop.yarn; - -import "Security.proto"; -import "yarn_protos.proto"; - -message TajoContainerIdProto { - optional ApplicationIdProto app_id = 1; - optional ApplicationAttemptIdProto app_attempt_id = 2; - optional int32 id = 3; -} - -message TajoContainerProto { - optional TajoContainerIdProto id = 1; - optional NodeIdProto nodeId = 2; - optional string node_http_address = 3; - optional ResourceProto resource = 4; - optional PriorityProto priority = 5; - optional hadoop.common.TokenProto container_token = 6; -} diff --git a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto index 9be0189be3..ee215659ef 100644 --- a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto +++ b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto @@ -28,7 +28,6 @@ import "tajo_protos.proto"; import "TajoIdProtos.proto"; import "CatalogProtos.proto"; import "PrimitiveProtos.proto"; -import "ContainerProtocol.proto"; package hadoop.yarn; diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto index dc6fc692e1..c0172b1e47 100644 --- a/tajo-core/src/main/proto/QueryMasterProtocol.proto +++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto @@ -27,7 +27,6 @@ import "TajoIdProtos.proto"; import "CatalogProtos.proto"; import "PrimitiveProtos.proto"; import "TajoWorkerProtocol.proto"; -import "ContainerProtocol.proto"; import "QueryCoordinatorProtocol.proto"; package hadoop.yarn; diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto index c9ef067eed..89871bc67b 100644 --- a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto +++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto @@ -23,7 +23,6 @@ option java_generic_services = true; option java_generate_equals_and_hash = true; import "QueryCoordinatorProtocol.proto"; -import "ContainerProtocol.proto"; import "tajo_protos.proto"; import "TajoIdProtos.proto"; diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index e210c5d85a..17626894fb 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -28,7 +28,6 @@ import "TajoIdProtos.proto"; import "CatalogProtos.proto"; import "PrimitiveProtos.proto"; import "Plan.proto"; -import "ContainerProtocol.proto"; import "QueryCoordinatorProtocol.proto"; package hadoop.yarn; diff --git a/tajo-core/src/main/resources/webapps/admin/cluster.jsp b/tajo-core/src/main/resources/webapps/admin/cluster.jsp index 97ca698509..bcda1359ba 100644 --- a/tajo-core/src/main/resources/webapps/admin/cluster.jsp +++ b/tajo-core/src/main/resources/webapps/admin/cluster.jsp @@ -24,7 +24,6 @@ <%@ page import="org.apache.tajo.service.ServiceTracker" %> <%@ page import="org.apache.tajo.service.TajoMasterInfo" %> <%@ page import="org.apache.tajo.master.rm.Worker" %> -<%@ page import="org.apache.tajo.master.rm.WorkerResource" %> <%@ page import="org.apache.tajo.master.rm.WorkerState" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.TUtil" %> @@ -56,7 +55,7 @@ for(Worker eachWorker: workers.values()) { liveQueryMasters.add(eachWorker); liveWorkers.add(eachWorker); - runningQueryMasterTasks += eachWorker.getResource().getNumQueryMasterTasks(); + runningQueryMasterTasks += eachWorker.getNumRunningQueryMaster(); } for (Worker inactiveWorker : master.getContext().getResourceManager().getInactiveWorkers().values()) { @@ -169,12 +168,11 @@ } else { %> - + <% int no = 1; for(Worker queryMaster: liveQueryMasters) { - WorkerResource resource = queryMaster.getResource(); WorkerConnectionInfo connectionInfo = queryMaster.getConnectionInfo(); String queryMasterHttp = "http://" + connectionInfo.getHost() + ":" + connectionInfo.getHttpInfoPort() + "/index.jsp"; @@ -183,8 +181,7 @@ - - + @@ -233,11 +230,10 @@ } else { %>
NoQueryMasterClient PortRunning QueryHeap(free/total/max)HeartbeatStatus
NoQueryMasterClient PortRunning QueryHeartbeatStatus
<%=no++%> <%=connectionInfo.getHost() + ":" + connectionInfo.getQueryMasterPort()%> <%=connectionInfo.getClientPort()%><%=resource.getNumQueryMasterTasks()%><%=resource.getFreeHeap()/1024/1024%>/<%=resource.getTotalHeap()/1024/1024%>/<%=resource.getMaxHeap()/1024/1024%> MB<%=queryMaster.getNumRunningQueryMaster()%> <%=JSPUtil.getElapsedTime(queryMaster.getLastHeartbeatTime(), System.currentTimeMillis())%> <%=queryMaster.getState()%>
- + <% int no = 1; for(Worker worker: liveWorkers) { - WorkerResource resource = worker.getResource(); WorkerConnectionInfo connectionInfo = worker.getConnectionInfo(); String workerHttp = "http://" + connectionInfo.getHost() + ":" + connectionInfo.getHttpInfoPort() + "/index.jsp"; %> @@ -245,10 +241,9 @@ - - - - + + + @@ -277,7 +272,6 @@ <% int no = 1; for(Worker worker: deadWorkers) { - WorkerResource resource = worker.getResource(); %> diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp index e0cf876cbb..eb3b695c62 100644 --- a/tajo-core/src/main/resources/webapps/admin/index.jsp +++ b/tajo-core/src/main/resources/webapps/admin/index.jsp @@ -21,18 +21,15 @@ <%@ page import="org.apache.hadoop.fs.FileSystem" %> <%@ page import="org.apache.tajo.conf.TajoConf" %> -<%@ page import="org.apache.tajo.ipc.QueryCoordinatorProtocol" %> <%@ page import="org.apache.tajo.master.TajoMaster" %> <%@ page import="org.apache.tajo.service.ServiceTracker" %> <%@ page import="org.apache.tajo.service.TajoMasterInfo" %> -<%@ page import="org.apache.tajo.master.QueryInProgress" %> <%@ page import="org.apache.tajo.master.rm.Worker" %> <%@ page import="org.apache.tajo.master.rm.WorkerState" %> <%@ page import="org.apache.tajo.util.NetUtils" %> <%@ page import="org.apache.tajo.util.TUtil" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> <%@ page import="java.util.List" %> -<%@ page import="java.util.Collection" %> <%@ page import="java.util.Date" %> <%@ page import="java.util.Map" %> <%@ page import="java.net.InetSocketAddress" %> @@ -59,13 +56,10 @@ int runningQueryMasterTask = 0; - QueryCoordinatorProtocol.ClusterResourceSummary clusterResourceSummary = - master.getContext().getResourceManager().getClusterResourceSummary(); - for(Worker eachWorker: workers.values()) { numQueryMasters++; numLiveQueryMasters++; - runningQueryMasterTask += eachWorker.getResource().getNumQueryMasterTasks(); + runningQueryMasterTask += eachWorker.getNumRunningQueryMaster(); numWorkers++; numLiveWorkers++; } @@ -142,7 +136,7 @@

Cluster Summary

NoWorkerPullServer
Port
Running TasksMemory Resource
(used/total)
Disk Resource
(used/total)
Heap
(free/total/max)
HeartbeatStatus
NoWorkerPullServer
Port
Running TasksAvailableTotalHeartbeatStatus
<%=no++%> <%=connectionInfo.getHostAndPeerRpcPort()%> <%=connectionInfo.getPullServerPort()%><%=resource.getNumRunningTasks()%><%=resource.getUsedMemoryMB()%>/<%=resource.getMemoryMB()%><%=resource.getUsedDiskSlots()%>/<%=resource.getDiskSlots()%><%=resource.getFreeHeap()/1024/1024%>/<%=resource.getTotalHeap()/1024/1024%>/<%=resource.getMaxHeap()/1024/1024%> MB<%=worker.getNumRunningTasks()%><%=worker.getAvailableResource()%><%=worker.getTotalResourceCapability()%> <%=JSPUtil.getElapsedTime(worker.getLastHeartbeatTime(), System.currentTimeMillis())%> <%=worker.getState()%>
<%=no++%>
- + @@ -158,8 +152,8 @@ - - + + <% if (haService != null) { diff --git a/tajo-core/src/main/resources/webapps/admin/query_executor.jsp b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp index 1a58583627..49a43d45ba 100644 --- a/tajo-core/src/main/resources/webapps/admin/query_executor.jsp +++ b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp @@ -21,7 +21,6 @@ <%@ page import="org.apache.tajo.master.TajoMaster" %> <%@ page import="org.apache.tajo.service.ServiceTracker" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> -<%@ page import="javax.xml.ws.Service" %> <%@ page import="java.net.InetSocketAddress" %> <% @@ -335,7 +334,7 @@ function getPage() {

-
+
diff --git a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp index d08c0d9060..8449551608 100644 --- a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp @@ -204,7 +204,7 @@ %>
# Tasks: <%=numOfTasks%> / # Pages: <%=totalPage%>
TypeTotalLiveDeadRunning MasterMemory Resource
(used/total)
Disk Resource
(used/total)
TypeTotalLiveDeadRunning MasterAvailableTotal
Query Master <%=numQueryMasters%><%=numLiveWorkers%> <%=numDeadWorkersHtml%> -<%=clusterResourceSummary.getTotalMemoryMB() - clusterResourceSummary.getTotalAvailableMemoryMB()%>/<%=clusterResourceSummary.getTotalMemoryMB()%><%=clusterResourceSummary.getTotalDiskSlots() - clusterResourceSummary.getTotalAvailableDiskSlots()%>/<%=clusterResourceSummary.getTotalDiskSlots()%><%=master.getContext().getResourceManager().getScheduler().getClusterResource()%><%=master.getContext().getResourceManager().getScheduler().getMaximumResourceCapability()%>
- + <% int rowNo = (currentPage - 1) * pageSize + 1; for (TaskHistory eachTask: tasks) { @@ -226,12 +226,13 @@ %> - + - - - + + + + <% diff --git a/tajo-core/src/main/resources/webapps/admin/task.jsp b/tajo-core/src/main/resources/webapps/admin/task.jsp index 1530572a21..36e8075cb4 100644 --- a/tajo-core/src/main/resources/webapps/admin/task.jsp +++ b/tajo-core/src/main/resources/webapps/admin/task.jsp @@ -20,7 +20,6 @@ <%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%> <%@ page import="org.apache.tajo.util.JSPUtil" %> -<%@ page import="org.apache.tajo.util.TajoIdUtils" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> <%@ page import="java.text.SimpleDateFormat" %> <%@ page import="org.apache.tajo.master.TajoMaster" %> diff --git a/tajo-core/src/main/resources/webapps/worker/index.jsp b/tajo-core/src/main/resources/webapps/worker/index.jsp index bc3cb1e817..641b997f71 100644 --- a/tajo-core/src/main/resources/webapps/worker/index.jsp +++ b/tajo-core/src/main/resources/webapps/worker/index.jsp @@ -24,9 +24,7 @@ <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> <%@ page import="org.apache.tajo.worker.TajoWorker" %> -<%@ page import="org.apache.tajo.worker.TaskRunner" %> <%@ page import="java.text.SimpleDateFormat" %> -<%@ page import="java.util.ArrayList" %> <%@ page import="java.util.List" %> <% @@ -118,30 +116,6 @@ } //end of if %>
NoIdStatusProgressStartedRunning TimeHost
NoIdStatusProgressStartedRunning TimeRetryHost
<%=rowNo%><%=rowNo%> <%=eachTask.getId()%><%=eachTask.getState()%><%=JSPUtil.percentFormat(eachTask.getProgress())%>%<%=eachTask.getLaunchTime() == 0 ? "-" : df.format(eachTask.getLaunchTime())%><%=eachTask.getState()%><%=JSPUtil.percentFormat(eachTask.getProgress())%>%<%=eachTask.getLaunchTime() == 0 ? "-" : df.format(eachTask.getLaunchTime())%> <%=eachTask.getLaunchTime() == 0 ? "-" : eachTask.getRunningTime() + " ms"%><%=eachTask.getRetryCount()%> <%=taskHost%>
-

-


-<% - List taskRunners = new ArrayList(tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskRunners()); - JSPUtil.sortTaskRunner(taskRunners); -%> -

Running Task Containers

- [All Task Containers] -
- - - <% - for(TaskRunner eachTaskRunner: taskRunners) { - %> - - - - - - -<% - } //end of for -%> -
ContainerIdStartTimeFinishTimeRunTimeStatus
<%=eachTaskRunner.getId()%><%=df.format(eachTaskRunner.getStartTime())%><%=eachTaskRunner.getFinishTime() == 0 ? "-" : df.format(eachTaskRunner.getFinishTime())%><%=JSPUtil.getElapsedTime(eachTaskRunner.getStartTime(), eachTaskRunner.getFinishTime())%><%=eachTaskRunner.getServiceState()%>
\ No newline at end of file diff --git a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp index 92587188c8..44c2f805b0 100644 --- a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp @@ -229,15 +229,15 @@ "&page=" + currentPage + "&pageSize=" + pageSize + "&taskSeq=" + taskSeq + "&sort=" + sort + "&sortOrder=" + sortOrder; - String taskHost = eachTask.getSucceededHost() == null ? "-" : eachTask.getSucceededHost(); - if(eachTask.getSucceededHost() != null) { + String taskHost = eachTask.getSucceededWorker() == null ? "-" : eachTask.getSucceededWorker().getHost(); + if(eachTask.getSucceededWorker() != null) { TajoProtos.WorkerConnectionInfoProto worker = workerMap.get(eachTask.getLastAttempt().getWorkerConnectionInfo().getId()); if(worker != null) { TaskAttempt lastAttempt = eachTask.getLastAttempt(); if(lastAttempt != null) { TaskAttemptId lastAttemptId = lastAttempt.getId(); - taskHost = "" + eachTask.getSucceededHost() + ""; + taskHost = "" + eachTask.getSucceededWorker().getHost() + ""; } } } diff --git a/tajo-core/src/main/resources/webapps/worker/task.jsp b/tajo-core/src/main/resources/webapps/worker/task.jsp index 17e884afcf..32b16498c8 100644 --- a/tajo-core/src/main/resources/webapps/worker/task.jsp +++ b/tajo-core/src/main/resources/webapps/worker/task.jsp @@ -30,7 +30,6 @@ <%@ page import="org.apache.tajo.querymaster.Task" %> <%@ page import="org.apache.tajo.querymaster.Stage" %> <%@ page import="org.apache.tajo.storage.DataLocation" %> -<%@ page import="org.apache.tajo.storage.fragment.FileFragment" %> <%@ page import="org.apache.tajo.storage.fragment.FragmentConvertor" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.TajoIdUtils" %> @@ -161,7 +160,7 @@ Launch Time<%=task.getLaunchTime() == 0 ? "-" : df.format(task.getLaunchTime())%> Finish Time<%=task.getFinishTime() == 0 ? "-" : df.format(task.getFinishTime())%> Running Time<%=task.getLaunchTime() == 0 ? "-" : task.getRunningTime() + " ms"%> - Host<%=task.getSucceededHost() == null ? "-" : task.getSucceededHost()%> + Host<%=task.getSucceededWorker() == null ? "-" : task.getSucceededWorker().getHost()%> Shuffles# Shuffle Outputs: <%=numShuffles%>, Shuffle Key: <%=shuffleKey%>, Shuffle file: <%=shuffleFileName%> Data Locations<%=dataLocationInfos%> Fragment<%=fragmentInfo%> diff --git a/tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp b/tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp deleted file mode 100644 index bb5e90dbb3..0000000000 --- a/tajo-core/src/main/resources/webapps/worker/taskcontainers.jsp +++ /dev/null @@ -1,93 +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. - */ -%> -<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%> - -<%@ page import="org.apache.tajo.util.JSPUtil" %> -<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> -<%@ page import="org.apache.tajo.worker.TajoWorker" %> -<%@ page import="org.apache.tajo.worker.TaskRunner" %> -<%@ page import="java.text.SimpleDateFormat" %> -<%@ page import="java.util.ArrayList" %> -<%@ page import="java.util.List" %> -<%@ page import="org.apache.tajo.worker.TaskRunnerHistory" %> -<%@ page import="org.apache.tajo.worker.TaskRunnerHistory" %> - -<% - TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); - - List taskRunners = new ArrayList(tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskRunners()); - List histories = new ArrayList(tajoWorker.getWorkerContext().getTaskRunnerManager().getExecutionBlockHistories()); - - JSPUtil.sortTaskRunner(taskRunners); - JSPUtil.sortTaskRunnerHistory(histories); - - SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); -%> - - - - - - - tajo worker - - -<%@ include file="header.jsp"%> -
-

Tajo Worker: <%=tajoWorker.getWorkerContext().getWorkerName()%>

-
-

Running Task Containers

- - -<% - for(TaskRunner eachTaskRunner: taskRunners) { -%> - - - - - - -<% - } -%> -
ContainerIdStartTimeFinishTimeRunTimeStatus
<%=eachTaskRunner.getId()%><%=df.format(eachTaskRunner.getStartTime())%><%=eachTaskRunner.getFinishTime() == 0 ? "-" : df.format(eachTaskRunner.getFinishTime())%><%=JSPUtil.getElapsedTime(eachTaskRunner.getStartTime(), eachTaskRunner.getFinishTime())%><%=eachTaskRunner.getServiceState()%>
-

-


-

Finished Task Containers

- - -<% - for(TaskRunnerHistory history: histories) { - String taskRunnerId = TaskRunner.getId(history.getExecutionBlockId(), history.getContainerId()); -%> - - - - - - -<% - } -%> -
ContainerIdStartTimeFinishTimeRunTimeStatus
<%=taskRunnerId%><%=df.format(history.getStartTime())%><%=history.getFinishTime() == 0 ? "-" : df.format(history.getFinishTime())%><%=JSPUtil.getElapsedTime(history.getStartTime(), history.getFinishTime())%><%=history.getState()%>
-
- - \ No newline at end of file diff --git a/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp index 5c3ce7b302..ad147f6a3f 100644 --- a/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp +++ b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp @@ -32,31 +32,15 @@ <% TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); - String containerId = request.getParameter("containerId"); String quAttemptId = request.getParameter("taskAttemptId"); TaskAttemptId taskAttemptId = TajoIdUtils.parseTaskAttemptId(quAttemptId); - Task task = null; - TaskHistory taskHistory = null; - if(containerId == null || containerId.isEmpty() || "null".equals(containerId)) { - task = tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskByTaskAttemptId(taskAttemptId); - if (task != null) { - taskHistory = task.createTaskHistory(); - } else { - taskHistory = tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskHistoryByTaskAttemptId(taskAttemptId); - } + + TaskHistory taskHistory; + Task task = tajoWorker.getWorkerContext().getTaskManager().getTaskByTaskAttemptId(taskAttemptId); + if (task != null) { + taskHistory = task.createTaskHistory(); } else { - TaskRunner runner = tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskRunner(containerId); - if(runner != null) { - task = runner.getContext().getTask(taskAttemptId); - if (task != null) { - taskHistory = task.createTaskHistory(); - } else { - TaskRunnerHistory history = tajoWorker.getWorkerContext().getTaskRunnerManager().getExcutionBlockHistoryByTaskRunnerId(containerId); - if(history != null) { - taskHistory = history.getTaskHistory(taskAttemptId); - } - } - } + taskHistory = tajoWorker.getWorkerContext().getTaskManager().getTaskHistory(taskAttemptId.getTaskId()); } SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); %> diff --git a/tajo-core/src/main/resources/webapps/worker/tasks.jsp b/tajo-core/src/main/resources/webapps/worker/tasks.jsp deleted file mode 100644 index ab873cd4f9..0000000000 --- a/tajo-core/src/main/resources/webapps/worker/tasks.jsp +++ /dev/null @@ -1,107 +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. - */ -%> -<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%> - -<%@ page import="org.apache.tajo.TaskAttemptId" %> -<%@ page import="org.apache.tajo.util.JSPUtil" %> -<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> -<%@ page import="java.text.SimpleDateFormat" %> -<%@ page import="java.util.Map" %> -<%@ page import="org.apache.tajo.worker.*" %> - -<% - String containerId = request.getParameter("taskRunnerId"); - TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); - - TaskRunner taskRunner = tajoWorker.getWorkerContext().getTaskRunnerManager().getTaskRunner(containerId); - org.apache.tajo.worker.TaskRunnerHistory history = tajoWorker.getWorkerContext().getTaskRunnerManager().getExcutionBlockHistoryByTaskRunnerId(containerId); - SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); -%> - - - - - - tajo worker - <% - if (taskRunner == null && history == null) { - %> - - - - -<% - return; - } -%> - - -<%@ include file="header.jsp"%> -
-

Tajo Worker: <%=tajoWorker.getWorkerContext().getWorkerName()%>

-
-

Tasks

- - - <% - if (taskRunner != null) { - ExecutionBlockContext context = taskRunner.getContext(); - - for (Map.Entry entry : context.getTasks().entrySet()) { - TaskAttemptId taskAttemptId = entry.getKey(); - TaskHistory eachTask = entry.getValue().createTaskHistory(); - %> - - - - - - - - <% - } - } - - if (history != null) { - - - for (Map.Entry entry : history.getTaskHistoryMap().entrySet()) { - TaskAttemptId taskAttemptId = entry.getKey(); - TaskHistory eachTask = entry.getValue(); - %> - - - - - - - - <% - } - } - %> -
IdStartTimeFinishTimeRunTimeStatus
- <%=taskAttemptId%><%=df.format(eachTask.getStartTime())%><%=eachTask.getFinishTime() == 0 ? "-" : df.format(eachTask.getFinishTime())%><%=JSPUtil.getElapsedTime(eachTask.getStartTime(), eachTask.getFinishTime())%><%=eachTask.getState()%>
<%=taskAttemptId%><%=df.format(eachTask.getStartTime())%><%=eachTask.getFinishTime() == 0 ? "-" : df.format(eachTask.getFinishTime())%><%=JSPUtil.getElapsedTime(eachTask.getStartTime(), eachTask.getFinishTime())%><%=eachTask.getState()%>
-
- - \ No newline at end of file diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java index 76de201492..747d083dff 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -18,7 +18,6 @@ package org.apache.tajo.worker; -import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.statistics.TableStats; @@ -26,6 +25,7 @@ import org.apache.tajo.worker.event.TaskExecutorEvent; import java.io.IOException; +import java.util.List; import java.util.concurrent.Semaphore; public class MockTaskExecutor extends TaskExecutor { @@ -136,6 +136,16 @@ public TajoWorkerProtocol.TaskStatusProto getReport() { builder.setInputStats(new TableStats().getProto()); return builder.build(); } + + @Override + public TaskHistory createTaskHistory() { + return null; + } + + @Override + public List getFetchers() { + return null; + } }; } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index 72b8f64a57..b8b0378652 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -19,48 +19,32 @@ package org.apache.tajo.worker; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; -import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.*; -import org.apache.tajo.annotation.ThreadSafe; -import org.apache.tajo.catalog.CatalogService; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; -import org.apache.tajo.querymaster.QueryMaster; -import org.apache.tajo.querymaster.QueryMasterManagerService; -import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.service.ServiceTracker; -import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.history.HistoryReader; -import org.apache.tajo.util.history.HistoryWriter; -import org.apache.tajo.util.metrics.TajoSystemMetrics; -import org.apache.tajo.worker.event.ExecutionBlockStartEvent; -import org.apache.tajo.worker.event.ExecutionBlockStopEvent; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; import org.junit.After; import org.junit.Before; import org.junit.Test; import java.io.IOException; -import java.util.Queue; +import java.util.List; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class TestTaskExecutor { @@ -333,6 +317,16 @@ public TajoWorkerProtocol.TaskStatusProto getReport() { builder.setInputStats(new TableStats().getProto()); return builder.build(); } + + @Override + public TaskHistory createTaskHistory() { + return null; + } + + @Override + public List getFetchers() { + return null; + } }; } } From 4fab8787591302dace885417f4487976d586642d Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 18 Jun 2015 17:01:07 +0900 Subject: [PATCH 07/80] remove unused codes --- .../querymaster/DefaultTaskScheduler.java | 21 ------------------- .../apache/tajo/resource/NodeResource.java | 2 +- 2 files changed, 1 insertion(+), 22 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 4a94d6a3d9..3628e7ce3b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -857,27 +857,6 @@ public void assignToLeafTasks(LinkedList taskRequests) { TaskAttemptId attemptId = allocateLocalTask(host); if (attemptId == null) { // if a local task cannot be found - HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); - - if(hostVolumeMapping != null) { - if(!hostVolumeMapping.isRemote(attemptId)){ - // assign to remote volume - int volumeId = hostVolumeMapping.getAssignedVolumeId(attemptId); - hostVolumeMapping.decreaseConcurrency(volumeId); - hostVolumeMapping.increaseConcurrency(HostVolumeMapping.REMOTE); - } - // this part is remote concurrency management of a tail tasks - int tailLimit = Math.max(remainingScheduledObjectNum() / (leafTaskHostMapping.size() * 2), 1); - - if(hostVolumeMapping.getRemoteConcurrency() > tailLimit){ - //release container - int volumeId = hostVolumeMapping.getAssignedVolumeId(attemptId); - hostVolumeMapping.decreaseConcurrency(volumeId); - //taskRequest.getCallback().run(stopTaskRunnerReq); - continue; - } - } - ////////////////////////////////////////////////////////////////////// // rack-local allocation ////////////////////////////////////////////////////////////////////// diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java index f51fc07e12..c7fe55b5aa 100644 --- a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java @@ -183,6 +183,6 @@ public int compareTo(NodeResource other) { @Override public String toString() { - return ""; + return "(Memory:" + getMemory() + ", Disks:" + getDisks() + ", vCores:" + getVirtualCores() + ")"; } } From b82916043dccb26dd4dcb20f5ae70c1c202d5327 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 18 Jun 2015 17:38:46 +0900 Subject: [PATCH 08/80] add debug message --- .../org/apache/tajo/querymaster/DefaultTaskScheduler.java | 4 ++-- .../main/java/org/apache/tajo/worker/NodeResourceManager.java | 4 ++-- tajo-core/src/main/resources/webapps/worker/index.jsp | 2 ++ 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 3628e7ce3b..6281211498 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -276,7 +276,7 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { CallFuture callBack = new CallFuture(); QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); request.setCapacity(NodeResources.createResource(512).getProto()); - request.setNumContainers(scheduledObjectNum); + request.setNumContainers(Math.max(remainingScheduledObjectNum(), 1)); request.setPriority(stage.getPriority()); request.setQueryId(context.getMasterContext().getQueryId().getProto()); request.setQueue("default"); @@ -285,7 +285,7 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { request.setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()); request.addAllCandidateNodes(getWorkerIds(getLeafTaskHosts())); masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); - + LOG.info("request container:" + remainingScheduledObjectNum()); QueryCoordinatorProtocol.NodeResourceResponseProto responseProto = callBack.get(); for (QueryCoordinatorProtocol.AllocationResourceProto proto : responseProto.getResourceList()) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index 3ba4ef7c04..d14fce59bc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -111,11 +111,11 @@ public Dispatcher getDispatcher() { return dispatcher; } - protected NodeResource getTotalResource() { + public NodeResource getTotalResource() { return totalResource; } - protected NodeResource getAvailableResource() { + public NodeResource getAvailableResource() { return availableResource; } diff --git a/tajo-core/src/main/resources/webapps/worker/index.jsp b/tajo-core/src/main/resources/webapps/worker/index.jsp index 641b997f71..5729658d01 100644 --- a/tajo-core/src/main/resources/webapps/worker/index.jsp +++ b/tajo-core/src/main/resources/webapps/worker/index.jsp @@ -50,6 +50,8 @@ MaxHeap: <%=Runtime.getRuntime().maxMemory()/1024/1024%> MB TotalHeap: <%=Runtime.getRuntime().totalMemory()/1024/1024%> MB FreeHeap: <%=Runtime.getRuntime().freeMemory()/1024/1024%> MB + Available Resource: <%= tajoWorker.getWorkerContext().getNodeResourceManager().getAvailableResource() %> + Running Tasks: <%= tajoWorker.getWorkerContext().getTaskManager().getRunningTasks() %> Configuration:detail... Environment:detail... Threads:thread dump... From e5318b8030bbf1821aacbe0ddae45d94bc403afa Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 18 Jun 2015 18:23:07 +0900 Subject: [PATCH 09/80] add changable heartBeatInterval --- .../tajo/master/rm/TajoResourceTracker.java | 3 +++ .../scheduler/AbstractQueryScheduler.java | 7 ++---- .../master/scheduler/SimpleScheduler.java | 7 +++++- .../apache/tajo/worker/NodeStatusUpdater.java | 24 ++++++++++++------- .../main/proto/ResourceTrackerProtocol.proto | 3 ++- 5 files changed, 29 insertions(+), 15 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index 6b37a264e6..411f26a7c8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -189,6 +189,9 @@ public void nodeHeartbeat( } } } finally { + if(manager.getScheduler().getRunningQuery() > 0) { + response.setHeartBeatInterval(1000); //1 sec + } done.run(response.setCommand(responseCommand).build()); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java index 0fe614514e..deb742f09e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java @@ -18,13 +18,8 @@ package org.apache.tajo.master.scheduler; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; import org.apache.tajo.QueryId; -import org.apache.tajo.master.TajoMaster; -import org.apache.tajo.master.rm.TajoRMContext; import org.apache.tajo.resource.NodeResource; import org.apache.tajo.resource.NodeResources; import org.apache.tajo.resource.ResourceCalculator; @@ -57,6 +52,8 @@ public NodeResource getMaximumResourceCapability() { return maxResource; } + public abstract int getRunningQuery(); + public abstract ResourceCalculator getResourceCalculator(); public abstract void submitQuery(QuerySchedulingInfo schedulingInfo); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index ceb2414bf7..0b2846d427 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -63,7 +63,7 @@ public class SimpleScheduler extends AbstractQueryScheduler { private final Map assignedQueryMasterMap = Maps.newHashMap(); private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator(); - + private volatile int runningQuery; private final Thread queryProcessor; private QueueInfo queueInfo; @@ -126,6 +126,11 @@ protected void serviceStart() throws Exception { super.serviceStart(); } + @Override + public int getRunningQuery() { + return assignedQueryMasterMap.size(); + } + @Override public ResourceCalculator getResourceCalculator() { return resourceCalculator; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index edd6ed67c2..10bb7f63a4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -35,6 +35,7 @@ import org.apache.tajo.rpc.RpcConstants; import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.service.ServiceTrackerFactory; +import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.event.NodeStatusEvent; import java.net.ConnectException; @@ -57,7 +58,8 @@ public class NodeStatusUpdater extends AbstractService implements EventHandler heartBeatRequestQueue; private final TajoWorker.WorkerContext workerContext; private AsyncRpcClient rmClient; @@ -72,15 +74,14 @@ public NodeStatusUpdater(TajoWorker.WorkerContext workerContext) { @Override public void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - this.tajoConf = (TajoConf) conf; + + this.tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); this.heartBeatRequestQueue = Queues.newLinkedBlockingQueue(); this.serviceTracker = ServiceTrackerFactory.get(tajoConf); this.workerContext.getNodeResourceManager().getDispatcher().register(NodeStatusEvent.EventType.class, this); this.heartBeatInterval = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL); this.updaterThread = new StatusUpdaterThread(); + super.serviceInit(conf); } @@ -219,9 +220,16 @@ public void run() { if (lastResponse != null) { if (lastResponse.getCommand() == ResponseCommand.NORMAL) { List events = Lists.newArrayList(); + + if(lastResponse.hasHeartBeatInterval()) { + nextHeartBeatInterval = lastResponse.getHeartBeatInterval(); + } else { + nextHeartBeatInterval = heartBeatInterval; + } + try { /* batch update to ResourceTracker */ - drain(events, Math.max(queueingThreshold, 1), heartBeatInterval, TimeUnit.MILLISECONDS); + drain(events, Math.max(queueingThreshold, 1), nextHeartBeatInterval, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { break; } @@ -231,7 +239,7 @@ public void run() { lastResponse = sendHeartbeat(createResourceReport().build()); } else { // send ping; - lastResponse = sendHeartbeat(createHeartBeatReport().build()); + lastResponse = sendHeartbeat(createResourceReport().build()); } } else if (lastResponse.getCommand() == ResponseCommand.MEMBERSHIP) { @@ -255,7 +263,7 @@ public void run() { if (!isStopped) { synchronized (updaterThread) { try { - updaterThread.wait(heartBeatInterval); + updaterThread.wait(nextHeartBeatInterval); } catch (InterruptedException ie) { // Do Nothing } diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto index 89871bc67b..17589d4613 100644 --- a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto +++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto @@ -40,7 +40,8 @@ message NodeHeartbeatRequestProto { message NodeHeartbeatResponseProto { required ResponseCommand command = 1 [default = NORMAL]; - repeated QueryIdProto queryId = 2; + optional int32 heartBeatInterval = 2; + repeated QueryIdProto queryId = 3; } enum ResponseCommand { From 7ec50d034f9cd1bbc2fc7cce3eda35324681fee8 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 19 Jun 2015 12:40:06 +0900 Subject: [PATCH 10/80] TAJO-1397 --- .../tajo/master/rm/TajoResourceTracker.java | 10 ++--- .../master/scheduler/SimpleScheduler.java | 7 ++-- .../querymaster/DefaultTaskScheduler.java | 41 ++++++++++++------- .../apache/tajo/worker/NodeStatusUpdater.java | 17 ++------ 4 files changed, 38 insertions(+), 37 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index 411f26a7c8..b03b0e6951 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -128,13 +128,11 @@ public void nodeHeartbeat( if(rmContext.getWorkers().containsKey(workerId)) { // if worker is running - if (heartbeat.hasAvailableResource()) { - // status update - rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(heartbeat)); + // status update + rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(heartbeat)); - //refresh scheduler resource - rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); - } + //refresh scheduler resource + rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); // refresh ping workerLivelinessMonitor.receivedPing(workerId); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 0b2846d427..53ea332c63 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -181,9 +181,6 @@ public int getNumClusterNodes() { @Override public List reserve(QueryId queryId, QueryCoordinatorProtocol.NodeResourceRequestProto request) { - if (LOG.isDebugEnabled()) { - LOG.debug("Request:" + request.toString() + "Cluster resource: " + getClusterResource()); - } List reservedResources; NodeResource capacity = new NodeResource(request.getCapacity()); @@ -211,6 +208,10 @@ public int getNumClusterNodes() { randomWorkers, capacity, requiredContainers - reservedResources.size())); } + LOG.info("Request: " + request.getCapacity() + ", containerNum:"+ request.getNumContainers() + + "Current cluster resource: " + getClusterResource()); + if (LOG.isDebugEnabled()) { + } return reservedResources; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 6281211498..447f873418 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -296,7 +296,7 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { if(responseProto.getResourceCount() == 0) { synchronized (schedulingThread){ - schedulingThread.wait(50); + schedulingThread.wait(100); } } else { LOG.info("Allocates :" + responseProto.getResourceCount()); @@ -649,14 +649,19 @@ public int getAssignedVolumeId(TaskAttemptId attemptId) { } public void cancel(TaskAttempt taskAttempt) { - List locations = taskAttempt.getTask().getDataLocations(); - for (DataLocation location : locations) { - HostVolumeMapping volumeMapping = scheduledRequests.leafTaskHostMapping.get(location.getHost()); - volumeMapping.addTaskAttempt(location.getVolumeId(), taskAttempt); - } + if(taskAttempt.isLeafTask()) { + List locations = taskAttempt.getTask().getDataLocations(); + + for (DataLocation location : locations) { + HostVolumeMapping volumeMapping = scheduledRequests.leafTaskHostMapping.get(location.getHost()); + volumeMapping.addTaskAttempt(location.getVolumeId(), taskAttempt); + } - scheduledRequests.leafTasks.add(taskAttempt.getId()); + scheduledRequests.leafTasks.add(taskAttempt.getId()); + } else { + scheduledRequests.nonLeafTasks.add(taskAttempt.getId()); + } } private class ScheduledRequests { @@ -932,16 +937,22 @@ public void assignToLeafTasks(LinkedList taskRequests) { LOG.warn("cancel" + proto.getTaskRequest()); } - if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { - stage.getWorkerMap().put(connectionInfo.getId(), addr); - } if(responseProto.getCancellationTaskCount() > 0) { + if(requestProto.hasExecutionBlockRequest()) { + ebMap.remove(taskRequest.getWorkerId()); + } continue; } + } catch (Exception e) { LOG.error(e); } + + if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { + stage.getWorkerMap().put(connectionInfo.getId(), addr); + } + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); assignedRequest.add(attemptId); @@ -1042,14 +1053,16 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); LOG.info(responseProto.getCancellationTaskCount()); - if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { - stage.getWorkerMap().put(connectionInfo.getId(), addr); - } - if(responseProto.getCancellationTaskCount() > 0) { + if(requestProto.hasExecutionBlockRequest()) { + ebMap.remove(taskRequest.getWorkerId()); + } continue; } + if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { + stage.getWorkerMap().put(connectionInfo.getId(), addr); + } context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); //taskRequest.getCallback().run(taskAssign.getProto()); totalAssigned++; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 10bb7f63a4..04ed0c1d64 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -124,14 +124,9 @@ public int getQueueingThreshold() { return queueingThreshold; } - private NodeHeartbeatRequestProto.Builder createHeartBeatReport() { + private NodeHeartbeatRequestProto.Builder createResourceReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); - return requestProto; - } - - private NodeHeartbeatRequestProto.Builder createResourceReport() { - NodeHeartbeatRequestProto.Builder requestProto = createHeartBeatReport(); requestProto.setAvailableResource(workerContext.getNodeResourceManager().getAvailableResource().getProto()); requestProto.setRunningTasks(workerContext.getTaskManager().getRunningTasks()); return requestProto; @@ -260,14 +255,8 @@ public void run() { Runtime.getRuntime().halt(-1); } catch (Exception e) { LOG.error(e.getMessage(), e); - if (!isStopped) { - synchronized (updaterThread) { - try { - updaterThread.wait(nextHeartBeatInterval); - } catch (InterruptedException ie) { - // Do Nothing - } - } + if (isStopped) { + break; } } } From a8424871c2fb9a6f7af700f198ebe2f332852a80 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 19 Jun 2015 13:15:14 +0900 Subject: [PATCH 11/80] fix wrong QM address --- .../apache/tajo/querymaster/AbstractTaskScheduler.java | 2 +- .../apache/tajo/querymaster/DefaultTaskScheduler.java | 10 ++++++---- .../main/java/org/apache/tajo/querymaster/Stage.java | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java index eb8c93da3d..8ff0a8221e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java @@ -56,7 +56,7 @@ public int getTotalAssigned() { } public abstract void handleTaskRequestEvent(TaskRequestEvent event); - public abstract void releseTaskAttempt(TaskAttempt taskAttempt); + public abstract void releaseTaskAttempt(TaskAttempt taskAttempt); public abstract int remainingScheduledObjectNum(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 447f873418..cc592de36e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -360,13 +360,13 @@ public int size() { } } - public void releseTaskAttempt(TaskAttempt taskAttempt) { + public void releaseTaskAttempt(TaskAttempt taskAttempt) { if (taskAttempt.isLeafTask() && taskAttempt.getWorkerConnectionInfo() != null) { HostVolumeMapping mapping = scheduledRequests.leafTaskHostMapping.get(taskAttempt.getWorkerConnectionInfo().getHost()); if (mapping != null) { - int volumeId = mapping.lastAssignedVolumeId.remove(taskAttempt.getId()); + Integer volumeId = mapping.lastAssignedVolumeId.remove(taskAttempt.getId()); mapping.decreaseConcurrency(volumeId); } } @@ -913,7 +913,8 @@ public void assignToLeafTasks(LinkedList taskRequests) { TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); ebRequestProto.setExecutionBlockId(taskRequest.getExecutionBlockId().getProto()) - .setQueryMaster(connectionInfo.getProto()) + .setQueryMaster(context.getMasterContext() + .getQueryMasterContext().getWorkerContext().getConnectionInfo().getProto()) .setQueryContext(context.getMasterContext().getQueryContext().getProto()) .setQueryOutputPath(context.getMasterContext().getStagingDir().toString()) .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) @@ -1032,7 +1033,8 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); ebRequestProto.setExecutionBlockId(taskRequest.getExecutionBlockId().getProto()) - .setQueryMaster(connectionInfo.getProto()) + .setQueryMaster(context.getMasterContext() + .getQueryMasterContext().getWorkerContext().getConnectionInfo().getProto()) .setQueryContext(context.getMasterContext().getQueryContext().getProto()) .setQueryOutputPath(context.getMasterContext().getStagingDir().toString()) .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 858b30d0db..7a3201b67d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -1165,7 +1165,7 @@ public void transition(Stage stage, stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_FAILED)); } else { stage.completedTaskCount++; - stage.getTaskScheduler().releseTaskAttempt(task.getLastAttempt()); //FIXME + stage.getTaskScheduler().releaseTaskAttempt(task.getLastAttempt()); //FIXME if (taskEvent.getState() == TaskState.SUCCEEDED) { stage.succeededObjectCount++; From 50b78e6d4e4f6f9464f149cf3452d51115d9c22e Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 19 Jun 2015 13:28:30 +0900 Subject: [PATCH 12/80] TAJO-1397 --- .../tajo/querymaster/DefaultTaskScheduler.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index cc592de36e..e59fed3ef1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -265,7 +265,7 @@ private List getWorkerIds(Collection hosts){ @Override public void handleTaskRequestEvent(TaskRequestEvent event) { - + boolean isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); NettyClientBase tmClient = null; try { ServiceTracker serviceTracker = context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); @@ -275,12 +275,13 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { CallFuture callBack = new CallFuture(); QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); - request.setCapacity(NodeResources.createResource(512).getProto()); + request.setCapacity(NodeResources.createResource(1000, isLeaf ? 1 : 0).getProto()); request.setNumContainers(Math.max(remainingScheduledObjectNum(), 1)); request.setPriority(stage.getPriority()); request.setQueryId(context.getMasterContext().getQueryId().getProto()); request.setQueue("default"); - request.setType(QueryCoordinatorProtocol.ResourceType.LEAF); + request.setType(isLeaf ? QueryCoordinatorProtocol.ResourceType.LEAF: + QueryCoordinatorProtocol.ResourceType.INTERMEDIATE); request.setUserId("test"); request.setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()); request.addAllCandidateNodes(getWorkerIds(getLeafTaskHosts())); @@ -365,9 +366,8 @@ public void releaseTaskAttempt(TaskAttempt taskAttempt) { HostVolumeMapping mapping = scheduledRequests.leafTaskHostMapping.get(taskAttempt.getWorkerConnectionInfo().getHost()); - if (mapping != null) { - Integer volumeId = mapping.lastAssignedVolumeId.remove(taskAttempt.getId()); - mapping.decreaseConcurrency(volumeId); + if (mapping != null && mapping.lastAssignedVolumeId.containsKey(taskAttempt.getId())) { + mapping.decreaseConcurrency(mapping.lastAssignedVolumeId.remove(taskAttempt.getId())); } } } From a520494bd141da189074e1ea170efaadd3bbd3a0 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 19 Jun 2015 16:33:35 +0900 Subject: [PATCH 13/80] TAJO-1397 --- .../java/org/apache/tajo/conf/TajoConf.java | 5 +- .../master/scheduler/SimpleScheduler.java | 136 +----------------- .../querymaster/DefaultTaskScheduler.java | 4 +- .../apache/tajo/querymaster/QueryMaster.java | 2 + .../tajo/worker/NodeResourceManager.java | 40 ++++-- .../org/apache/tajo/worker/TaskExecutor.java | 15 +- .../org/apache/tajo/worker/TaskManager.java | 15 +- .../main/resources/webapps/worker/index.jsp | 30 ---- .../org/apache/tajo/TajoTestingCluster.java | 7 +- .../tajo/worker/TestNodeResourceManager.java | 1 + 10 files changed, 63 insertions(+), 192 deletions(-) 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 da6a529315..5093367187 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 @@ -204,7 +204,8 @@ public static enum ConfVars implements ConfigKey { PULLSERVER_PORT("tajo.pullserver.port", 0, Validators.range("0", "65535")), SHUFFLE_SSL_ENABLED_KEY("tajo.pullserver.ssl.enabled", false, Validators.bool()), SHUFFLE_FILE_FORMAT("tajo.shuffle.file-format", "RAW"), - SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM("tajo.shuffle.fetcher.parallel-execution.max-num", 2), + SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM("tajo.shuffle.fetcher.parallel-execution.max-num", + Runtime.getRuntime().availableProcessors() * 2, Validators.min("1")), SHUFFLE_FETCHER_CHUNK_MAX_SIZE("tajo.shuffle.fetcher.chunk.max-size", 8192), SHUFFLE_FETCHER_READ_TIMEOUT("tajo.shuffle.fetcher.read.timeout-sec", 120), SHUFFLE_FETCHER_READ_RETRY_MAX_NUM("tajo.shuffle.fetcher.read.retry.max-num", 20), @@ -317,7 +318,7 @@ public static enum ConfVars implements ConfigKey { $GROUPBY_MULTI_LEVEL_ENABLED("tajo.dist-query.groupby.multi-level-aggr", true), // WARN "tajo.yarn-rm.parallel-task-runner-launcher-num" should be set enough to avoid deadlock - $QUERY_EXECUTE_PARALLEL_MAX("tajo.query.execute.parallel.max", 1), + $QUERY_EXECUTE_PARALLEL_MAX("tajo.query.execute.parallel.max", 10), // for physical Executors $EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE("tajo.executor.external-sort.buffer-mb", 200L), diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 53ea332c63..5797749410 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -50,7 +50,6 @@ public class SimpleScheduler extends AbstractQueryScheduler { private static final Log LOG = LogFactory.getLog(SimpleScheduler.class); - private static final String DEFAULT_QUEUE_NAME = "default"; private static final Comparator COMPARATOR = new SchedulingAlgorithms.FifoComparator(); private volatile boolean isStopped = false; @@ -63,30 +62,20 @@ public class SimpleScheduler extends AbstractQueryScheduler { private final Map assignedQueryMasterMap = Maps.newHashMap(); private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator(); - private volatile int runningQuery; private final Thread queryProcessor; - private QueueInfo queueInfo; - private TajoConf tajoConf; public SimpleScheduler(TajoMaster.MasterContext context) { super(SimpleScheduler.class.getName()); this.masterContext = context; this.rmContext = context.getResourceManager().getRMContext(); - this.queueInfo = new SimpleQueue(); this.queryQueue = new PriorityBlockingQueue(11, COMPARATOR); this.queryProcessor = new Thread(new QueryProcessor()); } private void initScheduler(TajoConf conf) { - validateConf(conf); int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); this.minResource.setMemory(minQMMem).setVirtualCores(1); - this.queueInfo.setCapacity(1.0f); - this.queueInfo.setMaximumCapacity(queueInfo.getCapacity()); - this.queueInfo.setMaximumQueryCapacity(0.3f); // maximum parall - this.queueInfo.setQueueState(QueueState.RUNNING); - this.queueInfo.setChildQueues(new ArrayList()); updateResource(); this.queryProcessor.setName("Query Processor"); } @@ -103,14 +92,8 @@ private void updateResource() { NodeResources.update(maxResource, totalResource); NodeResources.update(clusterResource, resource); - if (getResourceCalculator().isInvalidDivisor(clusterResource)) { - this.queueInfo.setCurrentCapacity(0.0f); - } else { - this.queueInfo.setCurrentCapacity(getResourceCalculator().ratio(clusterResource, maxResource)); - } - - LOG.info("Scheduler resources \n current: " + getClusterResource() - + "\n maximum: " + getMaximumResourceCapability() + "\n queue: " + queueInfo); + LOG.info("Cluster Resource. available : " + getClusterResource() + + " maximum: " + getMaximumResourceCapability()); } @Override @@ -136,25 +119,6 @@ public ResourceCalculator getResourceCalculator() { return resourceCalculator; } - private void validateConf(TajoConf conf) { - // validate scheduler memory allocation setting - int minMem = conf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); - int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); - int maxMem = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); - - if (minMem <= 0 || minQMMem <= 0 || minMem + minQMMem > maxMem) { - throw new RuntimeException("Invalid resource scheduler memory" - + " allocation configuration" - + ", " + TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.varname - + "=" + minMem - + ", " + TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY.varname - + "=" + minQMMem - + ", " + TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname - + "=" + maxMem + ", min and max should be greater than 0" - + ", max should be no smaller than min."); - } - } - private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceRequest(QueryInfo queryInfo) { int qmMemory = tajoConf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); @@ -208,9 +172,9 @@ public int getNumClusterNodes() { randomWorkers, capacity, requiredContainers - reservedResources.size())); } - LOG.info("Request: " + request.getCapacity() + ", containerNum:"+ request.getNumContainers() - + "Current cluster resource: " + getClusterResource()); if (LOG.isDebugEnabled()) { + LOG.debug("Request: " + request.getCapacity() + ", containerNum:" + request.getNumContainers() + + "Current cluster resource: " + getClusterResource()); } return reservedResources; } @@ -259,6 +223,7 @@ private List reserveClusterResource(List worke public void handle(SchedulerEvent event) { switch (event.getType()) { case RESOURCE_RESERVE: + //TODO should consider request priority reserveResource(TUtil.checkTypeAndGet(event, ResourceReserveSchedulerEvent.class)); break; case RESOURCE_UPDATE: @@ -327,14 +292,11 @@ public void run() { e.printStackTrace(); break; } - - //QueueInfo queueInfo = getQueueInfo(query.getQueue(), true, true); - + //TODO get by assigned queue int maxAvailable = getResourceCalculator().computeAvailableContainers( getMaximumResourceCapability(), getMinimumResourceCapability()); - // limit maximum running queries -// if ((assignedQueryMasterMap.size() / maxAvailable) > queueInfo.getCurrentCapacity()) { + // check maximum running queries if (assignedQueryMasterMap.size() * 2 > maxAvailable) { queryQueue.add(query); synchronized (this) { @@ -381,88 +343,4 @@ public void run() { } } } - - static class SimpleQueue extends QueueInfo { - private List childQueues; - private float capacity; - private float currentCapacity; - private float maximumCapacity; - private float maximumQueryCapacity; - private QueueState state; - - @Override - public String getQueueName() { - return DEFAULT_QUEUE_NAME; - } - - @Override - public void setQueueName(String queueName) { - } - - @Override - public float getCapacity() { - return capacity; - } - - @Override - public void setCapacity(float capacity) { - this.capacity = capacity; - } - - @Override - public float getMaximumCapacity() { - return maximumCapacity; - } - - @Override - public void setMaximumCapacity(float maximumCapacity) { - this.maximumCapacity = maximumCapacity; - } - - @Override - public float getMaximumQueryCapacity() { - return maximumQueryCapacity; - } - - @Override - public void setMaximumQueryCapacity(float maximumQueryCapacity) { - this.maximumQueryCapacity = maximumQueryCapacity; - } - - @Override - public float getCurrentCapacity() { - return currentCapacity; - } - - @Override - public void setCurrentCapacity(float currentCapacity) { - this.currentCapacity = currentCapacity; - } - - @Override - public List getChildQueues() { - return childQueues; - } - - @Override - public void setChildQueues(List childQueues) { - this.childQueues = childQueues; - } - - @Override - public QueueState getQueueState() { - return state; - } - - @Override - public void setQueueState(QueueState queueState) { - this.state = queueState; - } - - @Override - public String toString() { - return String.format("Queue name: %s, state: %s, maximum: %f, current: %f", - getQueueName(), getQueueState(), getMaximumCapacity(), getCurrentCapacity()); - } - } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index e59fed3ef1..5033d187b2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -29,6 +29,7 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryIdFactory; import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; @@ -266,6 +267,7 @@ private List getWorkerIds(Collection hosts){ public void handleTaskRequestEvent(TaskRequestEvent event) { boolean isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); + int taskMem = context.getMasterContext().getConf().getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); NettyClientBase tmClient = null; try { ServiceTracker serviceTracker = context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); @@ -275,7 +277,7 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { CallFuture callBack = new CallFuture(); QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); - request.setCapacity(NodeResources.createResource(1000, isLeaf ? 1 : 0).getProto()); + request.setCapacity(NodeResources.createResource(taskMem, isLeaf ? 1 : 0).getProto()); request.setNumContainers(Math.max(remainingScheduledObjectNum(), 1)); request.setPriority(stage.getPriority()); request.setQueryId(context.getMasterContext().getQueryId().getProto()); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index a78bbe8d56..d1c6595191 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -217,6 +217,8 @@ public Collection getQueryMasterTasks() { return queryMasterTasks.values(); } + //This is not safe OOM + @Deprecated public Collection getFinishedQueryMasterTasks() { return finishedQueryMasterTasks.values(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index d14fce59bc..aa5352c445 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -43,6 +43,7 @@ public class NodeResourceManager extends AbstractService implements EventHandler private NodeResource totalResource; private NodeResource availableResource; private TajoConf tajoConf; + private boolean enableTest; public NodeResourceManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext) { super(NodeResourceManager.class.getName()); @@ -52,14 +53,12 @@ public NodeResourceManager(Dispatcher dispatcher, TajoWorker.WorkerContext worke @Override protected void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - this.tajoConf = (TajoConf)conf; + this.tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); this.totalResource = createWorkerResource(tajoConf); this.availableResource = NodeResources.clone(totalResource); this.dispatcher.register(NodeResourceEvent.EventType.class, this); - + validateConf(tajoConf); + this.enableTest = conf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE"); super.serviceInit(conf); LOG.info("Initialized NodeResourceManager for " + totalResource); } @@ -120,14 +119,19 @@ public NodeResource getAvailableResource() { } private boolean allocate(NodeResource resource) { - //TODO consider the jvm free memory - if (NodeResources.fitsIn(resource, availableResource)) { + + if (NodeResources.fitsIn(resource, availableResource) && checkFreeHeapMemory(resource)) { NodeResources.subtractFrom(availableResource, resource); return true; } return false; } + private boolean checkFreeHeapMemory(NodeResource resource) { + //TODO consider the jvm free memory + return true; + } + protected void startExecutionBlock(StartExecutionBlockRequestProto request) { workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new ExecutionBlockStartEvent(request)); } @@ -143,11 +147,12 @@ private void release(NodeResource resource) { private NodeResource createWorkerResource(TajoConf conf) { int memoryMb = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); - if (!conf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) { + if (!enableTest) { // Set memory resource to max heap int maxHeap = (int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB); if(maxHeap > memoryMb) { memoryMb = maxHeap; + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, memoryMb); } } @@ -162,4 +167,23 @@ private NodeResource createWorkerResource(TajoConf conf) { int diskParallels = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM); return NodeResource.createResource(memoryMb, disks * diskParallels, vCores); } + + private void validateConf(TajoConf conf) { + // validate node memory allocation setting + int minMem = conf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); + int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); + int maxMem = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); + + if (minMem <= 0 || minQMMem <= 0 || minMem + minQMMem > maxMem) { + throw new RuntimeException("Invalid resource worker memory" + + " allocation configuration" + + ", " + TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.varname + + "=" + minMem + + ", " + TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY.varname + + "=" + minQMMem + + ", " + TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname + + "=" + maxMem + ", min and max should be greater than 0" + + ", max should be no smaller than min."); + } + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 78644d11ee..dd5500a0b1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -51,7 +51,7 @@ public class TaskExecutor extends AbstractService implements EventHandler allocatedResourceMap; private final BlockingQueue taskQueue; private final AtomicInteger runningTasks; - private ThreadPoolExecutor fetcherExecutor; + private ExecutorService fetcherThreadPool; private ExecutorService threadPool; private TajoConf tajoConf; private volatile boolean isStopped; @@ -78,12 +78,9 @@ protected void serviceStart() throws Exception { this.threadPool = Executors.newFixedThreadPool(nThreads, new ThreadFactoryBuilder().setNameFormat("Task executor #%d").build()); - //TODO move to tajoConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); - int maxFetcherThreads = Runtime.getRuntime().availableProcessors() * 2; - this.fetcherExecutor = new ThreadPoolExecutor(Math.min(nThreads, maxFetcherThreads), - maxFetcherThreads, - 60L, TimeUnit.SECONDS, - new SynchronousQueue(true)); + int maxFetcherThreads = tajoConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + this.fetcherThreadPool = Executors.newFixedThreadPool(nThreads, + new ThreadFactoryBuilder().setNameFormat("Fetcher executor #%d").build()); for (int i = 0; i < nThreads; i++) { @@ -99,7 +96,7 @@ protected void serviceStop() throws Exception { isStopped = true; threadPool.shutdown(); - fetcherExecutor.shutdown(); + fetcherThreadPool.shutdown(); super.serviceStop(); } @@ -132,7 +129,7 @@ protected void stopTask(TaskAttemptId taskId) { } protected ExecutorService getFetcherExecutor() { - return fetcherExecutor; + return fetcherThreadPool; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index 875530b30f..756f07e733 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -29,13 +29,16 @@ import org.apache.tajo.TajoIdProtos; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; -import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.util.TUtil; -import org.apache.tajo.worker.event.*; +import org.apache.tajo.worker.event.ExecutionBlockStartEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeStatusEvent; +import org.apache.tajo.worker.event.TaskManagerEvent; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; /** * A TaskManager is responsible for managing executionBlock resource and tasks. @@ -47,8 +50,6 @@ public class TaskManager extends AbstractService implements EventHandler executionBlockContextMap; private final Dispatcher dispatcher; - private TajoConf tajoConf; - public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext) { super(TaskManager.class.getName()); @@ -59,8 +60,6 @@ public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext @Override protected void serviceInit(Configuration conf) throws Exception { - - this.tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); dispatcher.register(TaskManagerEvent.EventType.class, this); super.serviceInit(conf); } diff --git a/tajo-core/src/main/resources/webapps/worker/index.jsp b/tajo-core/src/main/resources/webapps/worker/index.jsp index 5729658d01..fa000128af 100644 --- a/tajo-core/src/main/resources/webapps/worker/index.jsp +++ b/tajo-core/src/main/resources/webapps/worker/index.jsp @@ -62,8 +62,6 @@ List queryMasterTasks = JSPUtil.sortQueryMasterTask(tajoWorker.getWorkerContext() .getQueryMasterManagerService().getQueryMaster().getQueryMasterTasks(), true); - List finishedQueryMasterTasks = JSPUtil.sortQueryMasterTask(tajoWorker.getWorkerContext() - .getQueryMasterManagerService().getQueryMaster().getFinishedQueryMasterTasks(), true); %>

Running Query

<% @@ -90,34 +88,6 @@ } //end of if %> -

-


-

Finished Query

- <% - if(finishedQueryMasterTasks.isEmpty()) { - out.write("No finished query master"); - } else { - %> - - - <% - for(QueryMasterTask eachQueryMasterTask: finishedQueryMasterTasks) { - Query query = eachQueryMasterTask.getQuery(); - long startTime = query != null ? query.getStartTime() : eachQueryMasterTask.getQuerySubmitTime(); - %> - - - - - - - - - <% - } //end of for - } //end of if - %> -
QueryIdStatusStartTimeFinishTimeProgressRunTime
<%=eachQueryMasterTask.getQueryId()%><%=eachQueryMasterTask.getState()%><%=df.format(startTime)%><%=(query == null || query.getFinishTime() == 0) ? "-" : df.format(query.getFinishTime())%><%=(query == null) ? "-" : (int)(query.getProgress()*100.0f)%>%<%=(query == null) ? "-" : JSPUtil.getElapsedTime(query.getStartTime(), query.getFinishTime())%>
\ No newline at end of file diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 8129ca6ebc..ca663997b0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -19,7 +19,6 @@ package org.apache.tajo; import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; import com.google.common.io.Closeables; import com.google.common.io.Files; import org.apache.commons.lang.StringUtils; @@ -40,7 +39,6 @@ import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanTestRuleProvider; import org.apache.tajo.master.TajoMaster; -import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.plan.rewrite.LogicalPlanTestRuleProvider; import org.apache.tajo.querymaster.Query; import org.apache.tajo.querymaster.QueryMasterTask; @@ -124,9 +122,8 @@ void initPropertiesAndConfigs() { conf.setClassVar(ConfVars.LOGICAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, LogicalPlanTestRuleProvider.class); conf.setClassVar(ConfVars.GLOBAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, GlobalPlanTestRuleProvider.class); - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 3072); - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 4); - + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 2000); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 2); // Client API RPC conf.setIntVar(ConfVars.RPC_CLIENT_WORKER_THREAD_NUM, 2); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 7fd4dd82a7..979d5e3154 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -234,6 +234,7 @@ public void testParallelRequest() throws Exception { // start parallel request ExecutorService executor = Executors.newFixedThreadPool(parallelCount); + List futureList = Lists.newArrayList(); long startTime = System.currentTimeMillis(); From 3983008808e3351ba723941095643feae458eb49 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sun, 21 Jun 2015 23:42:53 +0900 Subject: [PATCH 14/80] remove StartExecutionBlock event --- .../apache/tajo/engine/query/TaskRequest.java | 27 ++- .../tajo/engine/query/TaskRequestImpl.java | 45 +++-- .../master/cluster/WorkerConnectionInfo.java | 4 + .../querymaster/AbstractTaskScheduler.java | 1 + .../querymaster/DefaultTaskScheduler.java | 157 ++++++------------ .../QueryMasterManagerService.java | 33 ++++ .../tajo/querymaster/QueryMasterTask.java | 10 +- .../org/apache/tajo/querymaster/Stage.java | 2 +- .../tajo/worker/ExecutionBlockContext.java | 25 +-- .../tajo/worker/NodeResourceManager.java | 84 +++++----- .../apache/tajo/worker/NodeStatusUpdater.java | 2 + .../tajo/worker/TajoWorkerManagerService.java | 3 +- .../org/apache/tajo/worker/TaskExecutor.java | 69 ++++---- .../org/apache/tajo/worker/TaskManager.java | 103 +++++++++--- .../event/ExecutionBlockStartEvent.java | 35 ---- .../worker/event/ExecutionBlockStopEvent.java | 8 +- .../event/NodeResourceAllocateEvent.java | 2 +- .../event/NodeResourceDeallocateEvent.java | 8 +- .../tajo/worker/event/NodeResourceEvent.java | 18 +- .../worker/event/QMResourceAllocateEvent.java | 7 +- .../tajo/worker/event/TaskManagerEvent.java | 17 +- .../tajo/worker/event/TaskStartEvent.java | 16 +- .../src/main/proto/QueryMasterProtocol.proto | 1 + .../src/main/proto/TajoWorkerProtocol.proto | 35 ++-- .../tajo/querymaster/TestKillQuery.java | 28 ++-- .../tajo/worker/MockExecutionBlock.java | 4 +- .../tajo/worker/MockNodeResourceManager.java | 10 +- .../apache/tajo/worker/MockTaskExecutor.java | 4 +- .../apache/tajo/worker/MockTaskManager.java | 17 +- .../tajo/worker/TestNodeResourceManager.java | 26 +-- .../tajo/worker/TestNodeStatusUpdater.java | 4 +- .../apache/tajo/worker/TestTaskExecutor.java | 23 +-- .../apache/tajo/worker/TestTaskManager.java | 37 ++--- 33 files changed, 421 insertions(+), 444 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java index 2fa272a547..6429d4001b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java @@ -35,18 +35,17 @@ public interface TaskRequest extends ProtoObject { - public TaskAttemptId getId(); - public List getFragments(); - public String getOutputTableId(); - public boolean isClusteredOutput(); - public PlanProto.LogicalNodeTree getPlan(); - public boolean isInterQuery(); - public void setInterQuery(); - public void addFetch(String name, FetchImpl fetch); - public List getFetches(); - public boolean shouldDie(); - public void setShouldDie(); - public QueryContext getQueryContext(TajoConf conf); - public DataChannel getDataChannel(); - public Enforcer getEnforcer(); + String getQueryMasterHostAndPort(); + TaskAttemptId getId(); + List getFragments(); + String getOutputTableId(); + boolean isClusteredOutput(); + PlanProto.LogicalNodeTree getPlan(); + boolean isInterQuery(); + void setInterQuery(); + void addFetch(String name, FetchImpl fetch); + List getFetches(); + QueryContext getQueryContext(TajoConf conf); + DataChannel getDataChannel(); + Enforcer getEnforcer(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java index b4727dcff3..b9e154bd71 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java @@ -47,6 +47,7 @@ public class TaskRequestImpl implements TaskRequest { private QueryContext queryContext; private DataChannel dataChannel; private Enforcer enforcer; + private String queryMasterHostAndPort; private TaskRequestProto proto = TajoWorkerProtocol.TaskRequestProto.getDefaultInstance(); private TajoWorkerProtocol.TaskRequestProto.Builder builder = null; @@ -61,9 +62,9 @@ public TaskRequestImpl() { public TaskRequestImpl(TaskAttemptId id, List fragments, String outputTable, boolean clusteredOutput, PlanProto.LogicalNodeTree plan, QueryContext queryContext, DataChannel channel, - Enforcer enforcer) { + Enforcer enforcer, String queryMasterHostAndPort) { this(); - this.set(id, fragments, outputTable, clusteredOutput, plan, queryContext, channel, enforcer); + this.set(id, fragments, outputTable, clusteredOutput, plan, queryContext, channel, enforcer, queryMasterHostAndPort); } public TaskRequestImpl(TaskRequestProto proto) { @@ -75,7 +76,8 @@ public TaskRequestImpl(TaskRequestProto proto) { public void set(TaskAttemptId id, List fragments, String outputTable, boolean clusteredOutput, - PlanProto.LogicalNodeTree plan, QueryContext queryContext, DataChannel dataChannel, Enforcer enforcer) { + PlanProto.LogicalNodeTree plan, QueryContext queryContext, + DataChannel dataChannel, Enforcer enforcer, String queryMasterHostAndPort) { this.id = id; this.fragments = fragments; this.outputTable = outputTable; @@ -86,6 +88,7 @@ public void set(TaskAttemptId id, List fragments, this.queryContext = queryContext; this.dataChannel = dataChannel; this.enforcer = enforcer; + this.queryMasterHostAndPort = queryMasterHostAndPort; } @Override @@ -96,6 +99,19 @@ public TaskRequestProto getProto() { return proto; } + @Override + public String getQueryMasterHostAndPort() { + TaskRequestProtoOrBuilder p = viaProto ? proto : builder; + if (queryMasterHostAndPort != null) { + return this.queryMasterHostAndPort; + } + if (!p.hasQueryMasterHostAndPort()) { + return null; + } + this.queryMasterHostAndPort = p.getQueryMasterHostAndPort(); + return this.queryMasterHostAndPort; + } + @Override public TaskAttemptId getId() { TaskRequestProtoOrBuilder p = viaProto ? proto : builder; @@ -253,25 +269,6 @@ private void initFetches() { } } - @Override - public boolean shouldDie() { - TaskRequestProtoOrBuilder p = viaProto ? proto : builder; - if (shouldDie != null) { - return shouldDie; - } - if (!p.hasShouldDie()) { - return false; - } - this.shouldDie = p.getShouldDie(); - return this.shouldDie; - } - - @Override - public void setShouldDie() { - maybeInitBuilder(); - shouldDie = true; - } - private void maybeInitBuilder() { if (viaProto || builder == null) { builder = TajoWorkerProtocol.TaskRequestProto.newBuilder(proto); @@ -305,8 +302,8 @@ private void mergeLocalToBuilder() { builder.addFetches(fetches.get(i).getProto()); } } - if (this.shouldDie != null) { - builder.setShouldDie(this.shouldDie); + if (this.queryMasterHostAndPort != null) { + builder.setQueryMasterHostAndPort(this.queryMasterHostAndPort); } if (this.queryContext != null) { builder.setQueryContext(queryContext.getProto()); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/cluster/WorkerConnectionInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/cluster/WorkerConnectionInfo.java index 78d4978908..0159a21a01 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/cluster/WorkerConnectionInfo.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/cluster/WorkerConnectionInfo.java @@ -111,6 +111,10 @@ public String getHostAndPeerRpcPort() { return this.getHost() + ":" + this.getPeerRpcPort(); } + public String getHostAndQMPort() { + return this.getHost() + ":" + this.getQueryMasterPort(); + } + @Override public WorkerConnectionInfoProto getProto() { WorkerConnectionInfoProto.Builder builder = WorkerConnectionInfoProto.newBuilder(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java index 8ff0a8221e..b694d5e840 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java @@ -32,6 +32,7 @@ public abstract class AbstractTaskScheduler extends AbstractService implements E protected int hostLocalAssigned; protected int rackLocalAssigned; protected int totalAssigned; + protected int cancellation; protected Set leafTaskHosts = Sets.newHashSet(); /** diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 5033d187b2..71b087c987 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -26,11 +26,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.util.RackResolver; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.QueryIdFactory; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.TaskRequest; @@ -41,9 +38,7 @@ import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; import org.apache.tajo.master.event.TaskSchedulerEvent.EventType; -import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.serder.LogicalNodeSerializer; -import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.AsyncRpcClient; import org.apache.tajo.rpc.CallFuture; @@ -102,15 +97,14 @@ public void start() { this.schedulingThread = new Thread() { public void run() { - while(!stopEventHandling.get() && !Thread.currentThread().isInterrupted()) { - schedule(); -// try { -// } catch (InterruptedException e) { -// break; -// } catch (Throwable e) { -// LOG.fatal(e.getMessage(), e); -// break; -// } + while (!stopEventHandling.get() && !Thread.currentThread().isInterrupted()) { + + try { + schedule(); + } catch (Throwable e) { + LOG.fatal(e.getMessage(), e); + break; + } } LOG.info("TaskScheduler schedulingThread stopped"); } @@ -120,22 +114,6 @@ public void run() { super.start(); } - private static final TaskAttemptId NULL_ATTEMPT_ID; - public static final TajoWorkerProtocol.TaskRequestProto stopTaskRunnerReq; - static { - ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0); - NULL_ATTEMPT_ID = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, 0), 0); - - TajoWorkerProtocol.TaskRequestProto.Builder builder = - TajoWorkerProtocol.TaskRequestProto.newBuilder(); - builder.setId(NULL_ATTEMPT_ID.getProto()); - builder.setShouldDie(true); - builder.setOutputTable(""); - builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); - builder.setClusteredOutput(false); - stopTaskRunnerReq = builder.build(); - } - @Override public void stop() { if(stopEventHandling.getAndSet(true)){ @@ -270,21 +248,24 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { int taskMem = context.getMasterContext().getConf().getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); NettyClientBase tmClient = null; try { - ServiceTracker serviceTracker = context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); + ServiceTracker serviceTracker = + context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); tmClient = RpcClientManager.getInstance(). getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); QueryCoordinatorProtocol.QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); CallFuture callBack = new CallFuture(); - QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); + QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = + QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); request.setCapacity(NodeResources.createResource(taskMem, isLeaf ? 1 : 0).getProto()); request.setNumContainers(Math.max(remainingScheduledObjectNum(), 1)); request.setPriority(stage.getPriority()); request.setQueryId(context.getMasterContext().getQueryId().getProto()); - request.setQueue("default"); + //TODO set queue + request.setQueue(context.getMasterContext().getQueryContext().get("queue", "default")); request.setType(isLeaf ? QueryCoordinatorProtocol.ResourceType.LEAF: QueryCoordinatorProtocol.ResourceType.INTERMEDIATE); - request.setUserId("test"); + request.setUserId(context.getMasterContext().getQueryContext().getUser()); request.setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()); request.addAllCandidateNodes(getWorkerIds(getLeafTaskHosts())); masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); @@ -301,24 +282,11 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { synchronized (schedulingThread){ schedulingThread.wait(100); } - } else { - LOG.info("Allocates :" + responseProto.getResourceCount()); } } catch (Throwable e) { LOG.error(e.getMessage(), e); } - - - int hosts = scheduledRequests.leafTaskHostMapping.size(); - - // if available cluster resource are large then tasks, the scheduler thread are working immediately. -// if(remainingScheduledObjectNum() > 0 && -// (remainingScheduledObjectNum() <= hosts || hosts <= taskRequests.size())){ -// synchronized (schedulingThread){ -// schedulingThread.notifyAll(); -// } -// } } @Override @@ -820,6 +788,8 @@ public int compare(HostVolumeMapping v1, HostVolumeMapping v2) { public void assignToLeafTasks(LinkedList taskRequests) { Collections.shuffle(taskRequests); LinkedList remoteTaskRequests = new LinkedList(); + String queryMasterHostAndPort = context.getMasterContext().getQueryMasterContext().getWorkerContext(). + getConnectionInfo().getHostAndQMPort(); TaskRequestEvent taskRequest; while (leafTasks.size() > 0 && (!taskRequests.isEmpty() || !remoteTaskRequests.isEmpty())) { @@ -885,6 +855,7 @@ public void assignToLeafTasks(LinkedList taskRequests) { } } + if (attemptId != null) { Task task = stage.getTask(attemptId.getTaskId()); TaskRequest taskAssign = new TaskRequestImpl( @@ -894,40 +865,27 @@ public void assignToLeafTasks(LinkedList taskRequests) { false, LogicalNodeSerializer.serialize(task.getLogicalPlan()), context.getMasterContext().getQueryContext(), - stage.getDataChannel(), stage.getBlock().getEnforcer()); + stage.getDataChannel(), stage.getBlock().getEnforcer(), + queryMasterHostAndPort); + if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) { taskAssign.setInterQuery(); } - //taskRequest.getCallback().run(taskAssign.getProto()); - - - TajoWorkerProtocol.BatchAllocationRequestProto.Builder requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); + //TODO send batch request + TajoWorkerProtocol.BatchAllocationRequestProto.Builder + requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); requestProto.addTaskRequest(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() .setResource(taskRequest.getResponseProto().getResource()) .setTaskRequest(taskAssign.getProto()).build()); requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); - if(!ebMap.containsKey(taskRequest.getWorkerId())) { - // first request with starting ExecutionBlock - PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); - - TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); - ebRequestProto.setExecutionBlockId(taskRequest.getExecutionBlockId().getProto()) - .setQueryMaster(context.getMasterContext() - .getQueryMasterContext().getWorkerContext().getConnectionInfo().getProto()) - .setQueryContext(context.getMasterContext().getQueryContext().getProto()) - .setQueryOutputPath(context.getMasterContext().getStagingDir().toString()) - .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) - .setShuffleType(shuffleType); - requestProto.setExecutionBlockRequest(ebRequestProto.build()); - ebMap.put(taskRequest.getWorkerId(), true); - } CallFuture callFuture = new CallFuture(); - InetSocketAddress addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); + InetSocketAddress addr = stage.getWorkerMap().get(connectionInfo.getId()); + if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); + AsyncRpcClient tajoWorkerRpc = null; try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); @@ -935,15 +893,12 @@ public void assignToLeafTasks(LinkedList taskRequests) { tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); - for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { - cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); - LOG.warn("cancel" + proto.getTaskRequest()); - } - if(responseProto.getCancellationTaskCount() > 0) { - if(requestProto.hasExecutionBlockRequest()) { - ebMap.remove(taskRequest.getWorkerId()); + for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { + cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); + cancellation++; } + LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); continue; } @@ -951,14 +906,8 @@ public void assignToLeafTasks(LinkedList taskRequests) { LOG.error(e); } - - if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { - stage.getWorkerMap().put(connectionInfo.getId(), addr); - } - context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); assignedRequest.add(attemptId); - scheduledObjectNum--; } else { @@ -966,7 +915,7 @@ public void assignToLeafTasks(LinkedList taskRequests) { } } } - Map ebMap = Maps.newHashMap(); + private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) { if (masterPlan.isRoot(block)) { return false; @@ -982,6 +931,8 @@ private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) { public void assignToNonLeafTasks(LinkedList taskRequests) { Collections.shuffle(taskRequests); + String queryMasterHostAndPort = context.getMasterContext().getQueryMasterContext().getWorkerContext(). + getConnectionInfo().getHostAndQMPort(); TaskRequestEvent taskRequest; while (!taskRequests.isEmpty()) { @@ -999,6 +950,7 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { Task task; task = stage.getTask(attemptId.getTaskId()); + TaskRequest taskAssign = new TaskRequestImpl( attemptId, Lists.newArrayList(task.getAllFragments()), @@ -1007,7 +959,9 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { LogicalNodeSerializer.serialize(task.getLogicalPlan()), context.getMasterContext().getQueryContext(), stage.getDataChannel(), - stage.getBlock().getEnforcer()); + stage.getBlock().getEnforcer(), + queryMasterHostAndPort); + if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) { taskAssign.setInterQuery(); } @@ -1022,32 +976,20 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); - TajoWorkerProtocol.BatchAllocationRequestProto.Builder requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); + //TODO send batch request + TajoWorkerProtocol.BatchAllocationRequestProto.Builder + requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); requestProto.addTaskRequest(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() .setResource(taskRequest.getResponseProto().getResource()) .setTaskRequest(taskAssign.getProto()).build()); requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); - if(!ebMap.containsKey(taskRequest.getWorkerId())) { - // first request with starting ExecutionBlock - PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); - - TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); - ebRequestProto.setExecutionBlockId(taskRequest.getExecutionBlockId().getProto()) - .setQueryMaster(context.getMasterContext() - .getQueryMasterContext().getWorkerContext().getConnectionInfo().getProto()) - .setQueryContext(context.getMasterContext().getQueryContext().getProto()) - .setQueryOutputPath(context.getMasterContext().getStagingDir().toString()) - .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) - .setShuffleType(shuffleType); - requestProto.setExecutionBlockRequest(ebRequestProto.build()); - ebMap.put(taskRequest.getWorkerId(), true); - } CallFuture callFuture = new CallFuture(); - InetSocketAddress addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); + InetSocketAddress addr = stage.getWorkerMap().get(connectionInfo.getId()); + if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); + AsyncRpcClient tajoWorkerRpc = null; try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); @@ -1055,20 +997,17 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); - LOG.info(responseProto.getCancellationTaskCount()); if(responseProto.getCancellationTaskCount() > 0) { - if(requestProto.hasExecutionBlockRequest()) { - ebMap.remove(taskRequest.getWorkerId()); + for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { + cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); + cancellation++; } + LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); continue; } - if(!stage.getWorkerMap().containsKey(connectionInfo.getId())) { - stage.getWorkerMap().put(connectionInfo.getId(), addr); - } context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); - //taskRequest.getCallback().run(taskAssign.getProto()); totalAssigned++; scheduledObjectNum--; } catch (Exception e) { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index 8e0b580616..6594d7c43a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -26,11 +26,14 @@ import org.apache.hadoop.service.CompositeService; import org.apache.tajo.*; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.event.*; +import org.apache.tajo.plan.logical.LogicalNode; +import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.resource.NodeResource; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.rpc.CallFuture; @@ -180,6 +183,36 @@ public void doneExecutionBlock( done.run(TajoWorker.NULL_PROTO); } + @Override + public void getExecutionBlockContext(RpcController controller, + TajoWorkerProtocol.ExecutionBlockContextRequestProto request, + RpcCallback done) { + + QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask( + new QueryId(request.getExecutionBlockId().getQueryId())); + + Stage stage = queryMasterTask.getQuery().getStage(new ExecutionBlockId(request.getExecutionBlockId())); + + // first request with starting ExecutionBlock + PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); + + TajoWorkerProtocol.ExecutionBlockContextProto.Builder + ebRequestProto = TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); + ebRequestProto.setExecutionBlockId(request.getExecutionBlockId()) + .setQueryContext(stage.getContext().getQueryContext().getProto()) + .setQueryOutputPath(stage.getContext().getStagingDir().toString()) + .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) + .setShuffleType(shuffleType); + + //Set assigned worker to stage + if (!stage.getWorkerMap().containsKey(request.getWorker().getId())) { + stage.getWorkerMap().put(request.getWorker().getId(), + NetUtils.createSocketAddr(request.getWorker().getHost(), request.getWorker().getPeerRpcPort())); + } + + done.run(ebRequestProto.build()); + } + @Override public void killQuery(RpcController controller, TajoIdProtos.QueryIdProto request, RpcCallback done) { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index 6fc80eb6c3..a68d99ac64 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -68,6 +68,7 @@ import org.apache.tajo.util.metrics.TajoMetrics; import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter; import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; +import org.apache.tajo.worker.event.NodeResourceEvent; import org.apache.tajo.worker.event.NodeStatusEvent; import java.io.IOException; @@ -193,12 +194,13 @@ public void serviceStop() throws Exception { LOG.info("Stopping QueryMasterTask:" + queryId); //release QM resource - getQueryTaskContext().getQueryMasterContext().getWorkerContext().getNodeResourceManager().getDispatcher() - .getEventHandler().handle(new NodeResourceDeallocateEvent(allocation)); + EventHandler handler = getQueryTaskContext().getQueryMasterContext().getWorkerContext(). + getNodeResourceManager().getDispatcher().getEventHandler(); + + handler.handle(new NodeResourceDeallocateEvent(allocation, NodeResourceEvent.ResourceType.QUERY_MASTER)); //flush current node resource - getQueryTaskContext().getQueryMasterContext().getWorkerContext().getNodeResourceManager().getDispatcher() - .getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + handler.handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); if (!queryContext.getBool(SessionVars.DEBUG_ENABLED)) { cleanupQuery(getQueryId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 7a3201b67d..4da1c9a03c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -721,7 +721,7 @@ protected void stopExecutionBlock() { cleanupList = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); cleanupList.addAllExecutionBlockId(Lists.newArrayList(ebIds)); - stopRequest.setChild(cleanupList.build()); + stopRequest.setCleanupList(cleanupList.build()); stopRequest.setExecutionBlockId(getId().getProto()); sendStopExecutionBlockEvent(stopRequest.build()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index a850ad3589..779c8bb399 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -40,11 +40,8 @@ import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.storage.StorageUtil; -import org.apache.tajo.util.NetUtils; import java.io.IOException; -import java.net.ConnectException; -import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -53,8 +50,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; import static org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.Interface; +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; public class ExecutionBlockContext { /** class logger */ @@ -78,10 +75,9 @@ public class ExecutionBlockContext { private TajoQueryEngine queryEngine; private RpcClientManager connManager; - private InetSocketAddress qmMasterAddr; + private AsyncRpcClient queryMasterClient; private NettyClientBase client; private QueryMasterProtocol.QueryMasterProtocolService.Interface stub; - private WorkerConnectionInfo queryMaster; private TajoConf systemConf; // for the doAs block private UserGroupInformation taskOwner; @@ -97,11 +93,11 @@ public class ExecutionBlockContext { private final Map taskHistories = Maps.newTreeMap(); - public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, StartExecutionBlockRequestProto request) + public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, ExecutionBlockContextProto request, + AsyncRpcClient queryMasterClient) throws IOException { this.executionBlockId = new ExecutionBlockId(request.getExecutionBlockId()); this.connManager = RpcClientManager.getInstance(); - this.queryMaster = new WorkerConnectionInfo(request.getQueryMaster()); this.systemConf = workerContext.getConf(); this.reporter = new Reporter(); this.defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf); @@ -115,6 +111,7 @@ public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, StartExecut this.resource = new ExecutionBlockSharedResource(); this.workerContext = workerContext; this.shuffleType = request.getShuffleType(); + this.queryMasterClient = queryMasterClient; } public void init() throws Throwable { @@ -122,8 +119,6 @@ public void init() throws Throwable { LOG.info("Tajo Root Dir: " + systemConf.getVar(TajoConf.ConfVars.ROOT_DIR)); LOG.info("Worker Local Dir: " + systemConf.getVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR)); - this.qmMasterAddr = NetUtils.createSocketAddr(queryMaster.getHost(), queryMaster.getQueryMasterPort()); - LOG.info("QueryMaster Address:" + qmMasterAddr); UserGroupInformation.setConfiguration(systemConf); // TODO - 'load credential' should be implemented @@ -133,7 +128,7 @@ public void init() throws Throwable { // initialize DFS and LocalFileSystems this.taskOwner = taskOwner; - this.stub = getRpcClient().getStub(); + this.stub = queryMasterClient.getStub(); this.reporter.startReporter(); // resource intiailization try{ @@ -152,12 +147,8 @@ public ExecutionBlockSharedResource getSharedResource() { return resource; } - private NettyClientBase getRpcClient() - throws NoSuchMethodException, ConnectException, ClassNotFoundException { - if (client != null) return client; - - client = connManager.newClient(qmMasterAddr, QueryMasterProtocol.class, true); - return client; + private AsyncRpcClient getRpcClient() { + return queryMasterClient; } public Interface getStub() { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index aa5352c445..ba396ca0b3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -33,6 +33,8 @@ import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.event.*; +import java.util.concurrent.atomic.AtomicInteger; + import static org.apache.tajo.ipc.TajoWorkerProtocol.*; public class NodeResourceManager extends AbstractService implements EventHandler { @@ -40,6 +42,7 @@ public class NodeResourceManager extends AbstractService implements EventHandler private final Dispatcher dispatcher; private final TajoWorker.WorkerContext workerContext; + private final AtomicInteger runningQueryMasters = new AtomicInteger(0); private NodeResource totalResource; private NodeResource availableResource; private TajoConf tajoConf; @@ -66,43 +69,50 @@ protected void serviceInit(Configuration conf) throws Exception { @Override public void handle(NodeResourceEvent event) { - if (event instanceof QMResourceAllocateEvent) { - // allocate query master resource - QMResourceAllocateEvent allocateEvent = TUtil.checkTypeAndGet(event, QMResourceAllocateEvent.class); - NodeResource resource = new NodeResource(allocateEvent.getRequest().getResource()); - if (allocate(resource)) { - allocateEvent.getCallback().run(TajoWorker.TRUE_PROTO); - } else { - allocateEvent.getCallback().run(TajoWorker.FALSE_PROTO); - } - } else if (event instanceof NodeResourceAllocateEvent) { - // allocate task resource - NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; - BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); - for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { - NodeResource resource = new NodeResource(request.getResource()); - if (allocate(resource)) { - if (allocateEvent.getRequest().hasExecutionBlockRequest()) { - //send ExecutionBlock start event to TaskManager - startExecutionBlock(allocateEvent.getRequest().getExecutionBlockRequest()); + switch (event.getType()) { + case ALLOCATE: { + if (event.getResourceType() == NodeResourceEvent.ResourceType.TASK) { + // allocate task resource + NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; + BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); + for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { + NodeResource resource = new NodeResource(request.getResource()); + if (allocate(resource)) { + //send task start event to TaskExecutor + startTask(request.getTaskRequest(), resource); + } else { + // reject the exceeded requests + response.addCancellationTask(request); + } + } + allocateEvent.getCallback().run(response.build()); + + } else if (event.getResourceType() == NodeResourceEvent.ResourceType.QUERY_MASTER) { + QMResourceAllocateEvent allocateEvent = TUtil.checkTypeAndGet(event, QMResourceAllocateEvent.class); + // allocate query master resource + + NodeResource resource = new NodeResource(allocateEvent.getRequest().getResource()); + if (allocate(resource)) { + allocateEvent.getCallback().run(TajoWorker.TRUE_PROTO); + runningQueryMasters.incrementAndGet(); + } else { + allocateEvent.getCallback().run(TajoWorker.FALSE_PROTO); } - - //send task start event to TaskExecutor - startTask(request.getTaskRequest(), resource); - } else { - // reject the exceeded requests - response.addCancellationTask(request); } + break; } - allocateEvent.getCallback().run(response.build()); - - } else if (event instanceof NodeResourceDeallocateEvent) { - NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; - release(deallocateEvent.getResource()); + case DEALLOCATE: { + NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; + release(deallocateEvent.getResource()); - // send current resource to ResourceTracker - getDispatcher().getEventHandler().handle( - new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); + if (deallocateEvent.getResourceType() == NodeResourceEvent.ResourceType.QUERY_MASTER) { + runningQueryMasters.decrementAndGet(); + } + // send current resource to ResourceTracker + getDispatcher().getEventHandler().handle( + new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); + break; + } } } @@ -118,6 +128,10 @@ public NodeResource getAvailableResource() { return availableResource; } + public int getRunningQueryMasters() { + return runningQueryMasters.get(); + } + private boolean allocate(NodeResource resource) { if (NodeResources.fitsIn(resource, availableResource) && checkFreeHeapMemory(resource)) { @@ -132,10 +146,6 @@ private boolean checkFreeHeapMemory(NodeResource resource) { return true; } - protected void startExecutionBlock(StartExecutionBlockRequestProto request) { - workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new ExecutionBlockStartEvent(request)); - } - protected void startTask(TaskRequestProto request, NodeResource resource) { workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new TaskStartEvent(request, resource)); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 04ed0c1d64..bbf323cd83 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -129,6 +129,8 @@ private NodeHeartbeatRequestProto.Builder createResourceReport() { requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); requestProto.setAvailableResource(workerContext.getNodeResourceManager().getAvailableResource().getProto()); requestProto.setRunningTasks(workerContext.getTaskManager().getRunningTasks()); + requestProto.setRunningQueryMasters(workerContext.getNodeResourceManager().getRunningQueryMasters()); + return requestProto; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index ff3a09dfe1..1910c1b5e6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -24,7 +24,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; -import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryId; import org.apache.tajo.TajoIdProtos; import org.apache.tajo.TaskAttemptId; @@ -118,7 +117,7 @@ public void stopExecutionBlock(RpcController controller, try { workerContext.getTaskManager().getDispatcher().getEventHandler().handle( - new ExecutionBlockStopEvent(requestProto.getExecutionBlockId(), requestProto.getChild())); + new ExecutionBlockStopEvent(requestProto.getExecutionBlockId(), requestProto.getCleanupList())); done.run(TajoWorker.TRUE_PROTO); } catch (Exception e) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index dd5500a0b1..c4f243201c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -33,18 +33,21 @@ import org.apache.tajo.resource.NodeResource; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; -import org.apache.tajo.worker.event.TaskExecutorEvent; +import org.apache.tajo.worker.event.NodeResourceEvent; import org.apache.tajo.worker.event.TaskStartEvent; import java.io.IOException; import java.util.Map; -import java.util.concurrent.*; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; /** * TaskExecutor uses a number of threads equal to the number of slots available for running tasks on the Worker */ -public class TaskExecutor extends AbstractService implements EventHandler { +public class TaskExecutor extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskExecutor.class); private final TajoWorker.WorkerContext workerContext; @@ -68,7 +71,6 @@ public TaskExecutor(TajoWorker.WorkerContext workerContext) { protected void serviceInit(Configuration conf) throws Exception { this.tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); - this.workerContext.getTaskManager().getDispatcher().register(TaskExecutorEvent.EventType.class, this); super.serviceInit(conf); } @@ -124,8 +126,8 @@ protected Task getNextTask() { @SuppressWarnings("unchecked") protected void stopTask(TaskAttemptId taskId) { runningTasks.decrementAndGet(); - workerContext.getNodeResourceManager().getDispatcher().getEventHandler() - .handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId))); + workerContext.getNodeResourceManager().getDispatcher().getEventHandler().handle( + new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId), NodeResourceEvent.ResourceType.TASK)); } protected ExecutorService getFetcherExecutor() { @@ -149,37 +151,34 @@ protected Task createTask(ExecutionBlockContext executionBlockContext, } @Override - public void handle(TaskExecutorEvent event) { - - if (event instanceof TaskStartEvent) { - TaskStartEvent startEvent = (TaskStartEvent) event; - allocatedResourceMap.put(startEvent.getTaskId(), startEvent.getAllocatedResource()); - - ExecutionBlockContext context = workerContext.getTaskManager().getExecutionBlockContext( - startEvent.getTaskId().getTaskId().getExecutionBlockId()); - - try { - Task task = createTask(context, startEvent.getTaskRequest()); - if (task != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Arrival task: " + task.getTaskContext().getTaskId() + - ", allocated resource: " + startEvent.getAllocatedResource()); - } - taskQueue.put(task); - runningTasks.incrementAndGet(); - context.getWorkerContext().getWorkerSystemMetrics() - .histogram("tasks", "running").update(runningTasks.get()); - } else { - LOG.warn("Release duplicate task resource: " + startEvent.getAllocatedResource()); - stopTask(startEvent.getTaskId()); - } - } catch (InterruptedException e) { - if (!isStopped) { - LOG.fatal(e.getMessage(), e); + public void handle(TaskStartEvent event) { + + allocatedResourceMap.put(event.getTaskAttemptId(), event.getAllocatedResource()); + + ExecutionBlockContext context = workerContext.getTaskManager().getExecutionBlockContext( + event.getTaskAttemptId().getTaskId().getExecutionBlockId()); + + try { + Task task = createTask(context, event.getTaskRequest()); + if (task != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Arrival task: " + task.getTaskContext().getTaskId() + + ", allocated resource: " + event.getAllocatedResource()); } - } catch (IOException e) { - stopTask(startEvent.getTaskId()); + taskQueue.put(task); + runningTasks.incrementAndGet(); + context.getWorkerContext().getWorkerSystemMetrics() + .histogram("tasks", "running").update(runningTasks.get()); + } else { + LOG.warn("Release duplicate task resource: " + event.getAllocatedResource()); + stopTask(event.getTaskAttemptId()); + } + } catch (InterruptedException e) { + if (!isStopped) { + LOG.fatal(e.getMessage(), e); } + } catch (IOException e) { + stopTask(event.getTaskAttemptId()); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index 756f07e733..aff9d4b0ac 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -29,16 +29,23 @@ import org.apache.tajo.TajoIdProtos; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; +import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.worker.event.ExecutionBlockStartEvent; -import org.apache.tajo.worker.event.ExecutionBlockStopEvent; -import org.apache.tajo.worker.event.NodeStatusEvent; -import org.apache.tajo.worker.event.TaskManagerEvent; +import org.apache.tajo.rpc.*; +import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.TUtil; +import org.apache.tajo.worker.event.*; + import java.io.IOException; +import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockContextRequestProto; +import static org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockContextProto; /** * A TaskManager is responsible for managing executionBlock resource and tasks. @@ -49,13 +56,19 @@ public class TaskManager extends AbstractService implements EventHandler executionBlockContextMap; private final Dispatcher dispatcher; + private TaskExecutor executor; - public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext) { + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext){ + this(dispatcher, workerContext, null); + } + + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, TaskExecutor executor) { super(TaskManager.class.getName()); this.dispatcher = dispatcher; this.workerContext = workerContext; this.executionBlockContextMap = Maps.newHashMap(); + this.executor = executor; } @Override @@ -82,17 +95,42 @@ protected TajoWorker.WorkerContext getWorkerContext() { return workerContext; } - public int getRunningTasks(){ + protected TaskExecutor getTaskExecutor() { + if (executor == null) { + executor = workerContext.getTaskExecuor(); + } + return executor; + } + + public int getRunningTasks() { return workerContext.getTaskExecuor().getRunningTasks(); } - protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.StartExecutionBlockRequestProto request) { + protected ExecutionBlockContext createExecutionBlock(ExecutionBlockId executionBlockId, + String queryMasterHostAndPort) { + + LOG.info("QueryMaster Address:" + queryMasterHostAndPort); + + AsyncRpcClient client = null; try { - ExecutionBlockContext context = new ExecutionBlockContext(getWorkerContext(), request); + InetSocketAddress address = NetUtils.createSocketAddr(queryMasterHostAndPort); + ExecutionBlockContextRequestProto.Builder request = ExecutionBlockContextRequestProto.newBuilder(); + request.setExecutionBlockId(executionBlockId.getProto()) + .setWorker(getWorkerContext().getConnectionInfo().getProto()); + + client = RpcClientManager.getInstance().newClient(address, QueryMasterProtocol.class, true); + QueryMasterProtocol.QueryMasterProtocolService.Interface stub = client.getStub(); + CallFuture callback = new CallFuture(); + stub.getExecutionBlockContext(callback.getController(), request.build(), callback); + + ExecutionBlockContextProto contextProto = + callback.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); + ExecutionBlockContext context = new ExecutionBlockContext(getWorkerContext(), contextProto, client); context.init(); return context; } catch (Throwable e) { + RpcClientManager.cleanup(client); LOG.fatal(e.getMessage(), e); throw new RuntimeException(e); } @@ -126,26 +164,37 @@ protected void stopExecutionBlock(ExecutionBlockContext context, @Override public void handle(TaskManagerEvent event) { - LOG.info("======================== Processing " + event.getExecutionBlockId() + " of type " + event.getType()); - - if (event instanceof ExecutionBlockStartEvent) { - - //receive event from NodeResourceManager - if(!executionBlockContextMap.containsKey(event.getExecutionBlockId())) { - ExecutionBlockContext context = createExecutionBlock(((ExecutionBlockStartEvent) event).getRequestProto()); - executionBlockContextMap.put(context.getExecutionBlockId(), context); - LOG.info("Running ExecutionBlocks: " + executionBlockContextMap.size() - + ", running tasks:" + getRunningTasks() + ", resource: " - + workerContext.getNodeResourceManager().getAvailableResource()); - } else { - LOG.warn("Already initialized ExecutionBlock: " + event.getExecutionBlockId()); + + if(LOG.isDebugEnabled()) { + LOG.debug("======================== Processing " + event + " of type " + event.getType()); + } + + switch (event.getType()) { + case TASK_START: { + //receive event from NodeResourceManager + TaskStartEvent taskStartEvent = TUtil.checkTypeAndGet(event, TaskStartEvent.class); + if (!executionBlockContextMap.containsKey(taskStartEvent.getExecutionBlockId())) { + ExecutionBlockContext context = createExecutionBlock(taskStartEvent.getExecutionBlockId(), + taskStartEvent.getTaskRequest().getQueryMasterHostAndPort()); + + executionBlockContextMap.put(context.getExecutionBlockId(), context); + LOG.info("Running ExecutionBlocks: " + executionBlockContextMap.size() + + ", running tasks:" + getRunningTasks() + ", availableResource: " + + workerContext.getNodeResourceManager().getAvailableResource()); + } + + getTaskExecutor().handle(taskStartEvent); + break; + } + case EB_STOP: { + //receive event from QueryMaster + ExecutionBlockStopEvent executionBlockStopEvent = TUtil.checkTypeAndGet(event, ExecutionBlockStopEvent.class); + workerContext.getNodeResourceManager().getDispatcher().getEventHandler() + .handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + stopExecutionBlock(executionBlockContextMap.remove(executionBlockStopEvent.getExecutionBlockId()), + ((ExecutionBlockStopEvent) event).getCleanupList()); + break; } - } else if (event instanceof ExecutionBlockStopEvent) { - //receive event from QueryMaster - workerContext.getNodeResourceManager().getDispatcher().getEventHandler() - .handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); - stopExecutionBlock(executionBlockContextMap.remove(event.getExecutionBlockId()), - ((ExecutionBlockStopEvent) event).getCleanupList()); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java deleted file mode 100644 index fa9a32d849..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStartEvent.java +++ /dev/null @@ -1,35 +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.worker.event; - -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.ipc.TajoWorkerProtocol; - -public class ExecutionBlockStartEvent extends TaskManagerEvent { - private TajoWorkerProtocol.StartExecutionBlockRequestProto requestProto; - - public ExecutionBlockStartEvent(TajoWorkerProtocol.StartExecutionBlockRequestProto requestProto) { - super(EventType.EB_START, new ExecutionBlockId(requestProto.getExecutionBlockId())); - this.requestProto = requestProto; - } - - public TajoWorkerProtocol.StartExecutionBlockRequestProto getRequestProto() { - return requestProto; - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java index 2b967ab1d8..75ccecab9c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java @@ -25,13 +25,19 @@ public class ExecutionBlockStopEvent extends TaskManagerEvent { private TajoWorkerProtocol.ExecutionBlockListProto cleanupList; + private ExecutionBlockId executionBlockId; public ExecutionBlockStopEvent(TajoIdProtos.ExecutionBlockIdProto executionBlockId, TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { - super(EventType.EB_STOP, new ExecutionBlockId(executionBlockId)); + super(EventType.EB_STOP); this.cleanupList = cleanupList; + this.executionBlockId = new ExecutionBlockId(executionBlockId); } public TajoWorkerProtocol.ExecutionBlockListProto getCleanupList() { return cleanupList; } + + public ExecutionBlockId getExecutionBlockId() { + return executionBlockId; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java index c0867e3e05..284db0e8dd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -31,7 +31,7 @@ public class NodeResourceAllocateEvent extends NodeResourceEvent { public NodeResourceAllocateEvent(BatchAllocationRequestProto request, RpcCallback callback) { - super(EventType.TASK_ALLOCATE); + super(EventType.ALLOCATE, ResourceType.TASK); this.callback = callback; this.request = request; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java index 31d9229ad4..d8841a2d17 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java @@ -25,12 +25,12 @@ public class NodeResourceDeallocateEvent extends NodeResourceEvent { private NodeResource resource; - public NodeResourceDeallocateEvent(TajoProtos.NodeResourceProto proto) { - this(new NodeResource(proto)); + public NodeResourceDeallocateEvent(TajoProtos.NodeResourceProto proto, ResourceType resourceType) { + this(new NodeResource(proto), resourceType); } - public NodeResourceDeallocateEvent(NodeResource resource) { - super(EventType.DEALLOCATE); + public NodeResourceDeallocateEvent(NodeResource resource, ResourceType resourceType) { + super(EventType.DEALLOCATE, resourceType); this.resource = resource; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java index 6f6fd9689a..c12551fcd0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java @@ -24,14 +24,24 @@ public class NodeResourceEvent extends AbstractEvent callback) { - super(EventType.QM_ALLOCATE); + super(EventType.ALLOCATE, ResourceType.QUERY_MASTER); this.callback = callback; this.request = request; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java index 39b541b097..efbaaf95c9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java @@ -19,25 +19,20 @@ package org.apache.tajo.worker.event; import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.TaskAttemptId; public class TaskManagerEvent extends AbstractEvent { // producer: NodeResourceManager, consumer: TaskManager public enum EventType { - EB_START, + TASK_START, + TASK_KILL, + TASK_ABORT, + EB_STOP } - private ExecutionBlockId executionBlockId; - public TaskManagerEvent(EventType eventType, - ExecutionBlockId executionBlockId) { - super(eventType); - this.executionBlockId = executionBlockId; - } - public ExecutionBlockId getExecutionBlockId() { - return executionBlockId; + public TaskManagerEvent(EventType eventType) { + super(eventType); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java index f60e7c4212..112afd1066 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java @@ -18,20 +18,24 @@ package org.apache.tajo.worker.event; +import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.resource.NodeResource; + import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; -public class TaskStartEvent extends TaskExecutorEvent { +public class TaskStartEvent extends TaskManagerEvent { private NodeResource allocatedResource; private TaskRequestProto taskRequest; + private TaskAttemptId taskAttemptId; public TaskStartEvent(TaskRequestProto taskRequest, NodeResource allocatedResource) { - super(EventType.START, new TaskAttemptId(taskRequest.getId())); + super(EventType.TASK_START); this.taskRequest = taskRequest; this.allocatedResource = allocatedResource; + this.taskAttemptId = new TaskAttemptId(taskRequest.getId()); } public NodeResource getAllocatedResource() { @@ -41,4 +45,12 @@ public NodeResource getAllocatedResource() { public TaskRequestProto getTaskRequest() { return taskRequest; } + + public TaskAttemptId getTaskAttemptId() { + return taskAttemptId; + } + + public ExecutionBlockId getExecutionBlockId() { + return taskAttemptId.getTaskId().getExecutionBlockId(); + } } diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto index c0172b1e47..33372b1380 100644 --- a/tajo-core/src/main/proto/QueryMasterProtocol.proto +++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto @@ -38,6 +38,7 @@ service QueryMasterProtocolService { rpc fatalError(TaskFatalErrorReport) returns (NullProto); rpc done (TaskCompletionReport) returns (NullProto); rpc doneExecutionBlock(ExecutionBlockReport) returns (NullProto); + rpc getExecutionBlockContext(ExecutionBlockContextRequestProto) returns (ExecutionBlockContextProto); //from TajoMaster's QueryJobManager rpc killQuery(QueryIdProto) returns (NullProto); diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 17626894fb..2d8e55b324 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -66,14 +66,14 @@ message TaskFatalErrorReport { } message TaskRequestProto { - required TaskAttemptIdProto id = 1; - repeated FragmentProto fragments = 2; - required string outputTable = 3; - required bool clusteredOutput = 4; - required LogicalNodeTree plan = 5; - optional bool interQuery = 6 [default = false]; - repeated FetchProto fetches = 7; - optional bool shouldDie = 8; + required string queryMasterHostAndPort = 1; + required TaskAttemptIdProto id = 2; + repeated FragmentProto fragments = 3; + required string outputTable = 4; + required bool clusteredOutput = 5; + required LogicalNodeTree plan = 6; + optional bool interQuery = 7 [default = false]; + repeated FetchProto fetches = 8; optional KeyValueSetProto queryContext = 9; optional DataChannelProto dataChannel = 10; optional EnforcerProto enforcer = 11; @@ -178,20 +178,24 @@ message DataChannelProto { optional string storeType = 10; } -message StartExecutionBlockRequestProto { + +message ExecutionBlockContextProto { required ExecutionBlockIdProto executionBlockId = 1; - required WorkerConnectionInfoProto queryMaster = 2; + optional string queryOutputPath = 2; - optional string queryOutputPath = 3; + required KeyValueSetProto queryContext = 3; + required string planJson = 4; + required ShuffleType shuffleType = 5; +} - required KeyValueSetProto queryContext = 4; - required string planJson = 5; - required ShuffleType shuffleType = 6; +message ExecutionBlockContextRequestProto { + required ExecutionBlockIdProto executionBlockId = 1; + required WorkerConnectionInfoProto worker = 2; } message StopExecutionBlockRequestProto { required ExecutionBlockIdProto executionBlockId = 1; - optional ExecutionBlockListProto child = 2; + optional ExecutionBlockListProto cleanupList = 2; } message ExecutionBlockListProto { @@ -206,7 +210,6 @@ message TaskAllocationRequestProto { message BatchAllocationRequestProto { required ExecutionBlockIdProto executionBlockId = 1; repeated TaskAllocationRequestProto taskRequest = 2; - optional StartExecutionBlockRequestProto executionBlockRequest = 3; } message BatchAllocationResponseProto { diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index 5935dcfe92..3569e2afe2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -233,20 +233,21 @@ public void testKillTask() throws Throwable { TaskId tid = QueryIdFactory.newTaskId(eid); final TajoConf conf = new TajoConf(); TaskRequestImpl taskRequest = new TaskRequestImpl(); + WorkerConnectionInfo queryMaster = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); TaskAttemptId attemptId = new TaskAttemptId(tid, 1); taskRequest.set(attemptId, new ArrayList(), - null, false, PlanProto.LogicalNodeTree.newBuilder().build(), new QueryContext(conf), null, null); + null, false, PlanProto.LogicalNodeTree.newBuilder().build(), new QueryContext(conf), + null, null, queryMaster.getHostAndQMPort()); taskRequest.setInterQuery(); - WorkerConnectionInfo queryMaster = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder - requestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); - requestProto.setExecutionBlockId(eid.getProto()) - .setQueryMaster(queryMaster.getProto()) - .setQueryContext(new QueryContext(conf).getProto()) + TajoWorkerProtocol.ExecutionBlockContextProto.Builder requestProtoBuilder = + TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); + requestProtoBuilder.setExecutionBlockId(eid.getProto()) .setPlanJson("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setQueryOutputPath("testpath") .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); TajoWorker.WorkerContext workerContext = new MockWorkerContext() { @@ -271,13 +272,12 @@ public NodeResourceManager getNodeResourceManager() { } }; - ExecutionBlockContext context = - new ExecutionBlockContext(workerContext, requestProto.build()) { - @Override - public Path createBaseDir() throws IOException { - return new Path("test"); - } - }; + ExecutionBlockContext context = new MockExecutionBlock(workerContext, requestProtoBuilder.build()) { + @Override + public Path createBaseDir() throws IOException { + return new Path("test"); + } + }; org.apache.tajo.worker.Task task = new TaskImpl(taskRequest, context, null); task.kill(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java index 1ca7b92a90..8abcb8f95c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java @@ -26,8 +26,8 @@ public class MockExecutionBlock extends ExecutionBlockContext { public MockExecutionBlock(TajoWorker.WorkerContext workerContext, - TajoWorkerProtocol.StartExecutionBlockRequestProto request) throws IOException { - super(workerContext, request); + TajoWorkerProtocol.ExecutionBlockContextProto request) throws IOException { + super(workerContext, request, null); } @Override diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java index b547916ba6..75c89bbb71 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java @@ -19,7 +19,6 @@ package org.apache.tajo.worker; import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryIdFactory; import org.apache.tajo.TaskAttemptId; @@ -49,13 +48,6 @@ public void handle(NodeResourceEvent event) { barrier.release(); } - @Override - protected void startExecutionBlock(TajoWorkerProtocol.StartExecutionBlockRequestProto request) { - if(enableTaskHandlerEvent) { - super.startExecutionBlock(request); - } - } - @Override protected void startTask(TajoWorkerProtocol.TaskRequestProto request, NodeResource resource) { if(enableTaskHandlerEvent) { @@ -80,8 +72,8 @@ protected static Queue createTask TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(ebId, i), 0); TajoWorkerProtocol.TaskRequestProto.Builder builder = TajoWorkerProtocol.TaskRequestProto.newBuilder(); + builder.setQueryMasterHostAndPort("localhost:0"); builder.setId(taskAttemptId.getProto()); - builder.setShouldDie(true); builder.setOutputTable(""); builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); builder.setClusteredOutput(false); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java index 747d083dff..90e752093a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -22,7 +22,7 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.worker.event.TaskExecutorEvent; +import org.apache.tajo.worker.event.TaskStartEvent; import java.io.IOException; import java.util.List; @@ -38,7 +38,7 @@ public MockTaskExecutor(Semaphore barrier, TajoWorker.WorkerContext workerContex } @Override - public void handle(TaskExecutorEvent event) { + public void handle(TaskStartEvent event) { super.handle(event); barrier.release(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java index ea82175db3..cad8063df2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java @@ -19,9 +19,11 @@ package org.apache.tajo.worker; import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.worker.event.TaskManagerEvent; import java.io.IOException; @@ -37,9 +39,16 @@ public MockTaskManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.Work } @Override - protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.StartExecutionBlockRequestProto request) { + protected ExecutionBlockContext createExecutionBlock(ExecutionBlockId executionBlockId, String queryMaster) { try { - return new MockExecutionBlock(getWorkerContext(), request); + TajoWorkerProtocol.ExecutionBlockContextProto.Builder builder = + TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); + builder.setExecutionBlockId(executionBlockId.getProto()) + .setPlanJson("test") + .setQueryContext(new QueryContext(new TajoConf()).getProto()) + .setQueryOutputPath("testpath") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + return new MockExecutionBlock(getWorkerContext(), builder.build()); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 979d5e3154..947773f72c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -22,26 +22,26 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; -import org.apache.tajo.*; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; -import org.junit.*; +import org.apache.tajo.worker.event.NodeResourceEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; import java.util.List; import java.util.Queue; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.*; - import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.junit.Assert.*; public class TestNodeResourceManager { private MockNodeResourceManager resourceManager; @@ -194,7 +194,8 @@ public void testNodeResourceDeallocateEvent() throws Exception { //deallocate for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { // direct invoke handler for testing - resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); + resourceManager.handle(new NodeResourceDeallocateEvent( + allocationRequestProto.getResource(), NodeResourceEvent.ResourceType.TASK)); } assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); @@ -213,20 +214,11 @@ public void testParallelRequest() throws Exception { final Queue totalTasks = MockNodeResourceManager.createTaskRequests(ebId, taskMemory, taskSize); - // first request with starting ExecutionBlock - TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); - ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setQueryContext(new QueryContext(conf).getProto()) - .setPlanJson("test") - .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); TaskAllocationRequestProto task = totalTasks.poll(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); requestProto.addTaskRequest(task); requestProto.setExecutionBlockId(ebId.getProto()); - requestProto.setExecutionBlockRequest(ebRequestProto.build()); CallFuture callFuture = new CallFuture(); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); assertTrue(callFuture.get().getCancellationTaskCount() == 0); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 968cf9616a..51a7771c8b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -145,7 +145,9 @@ public void testPing() throws Exception { TajoResourceTrackerProtocol.NodeHeartbeatRequestProto lastRequest = resourceTracker.getLastRequest(); assertTrue(lastRequest.hasWorkerId()); - assertFalse(lastRequest.hasAvailableResource()); + assertTrue(lastRequest.hasAvailableResource()); + assertTrue(lastRequest.hasRunningTasks()); + assertTrue(lastRequest.hasRunningQueryMasters()); assertFalse(lastRequest.hasTotalResource()); assertFalse(lastRequest.hasConnectionInfo()); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index b8b0378652..ae60d2076d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -24,10 +24,8 @@ import org.apache.tajo.*; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; @@ -42,7 +40,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; +import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -140,21 +139,12 @@ public void tearDown() { public void testTaskRequest() throws Exception { int requestSize = 1; - StartExecutionBlockRequestProto.Builder - ebRequestProto = StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); - ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setQueryContext(new QueryContext(conf).getProto()) - .setPlanJson("test") - .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); - CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); requestProto.setExecutionBlockId(ebId.getProto()); - requestProto.setExecutionBlockRequest(ebRequestProto.build()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); @@ -177,21 +167,12 @@ public void testTaskRequest() throws Exception { public void testTaskException() throws Exception { int requestSize = 1; - StartExecutionBlockRequestProto.Builder - ebRequestProto = StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); - ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setQueryContext(new QueryContext(conf).getProto()) - .setPlanJson("test") - .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); - CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); requestProto.setExecutionBlockId(ebId.getProto()); - requestProto.setExecutionBlockRequest(ebRequestProto.build()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index e43aa2f1b4..73474cfcee 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -21,22 +21,25 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; -import org.apache.tajo.*; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.LocalTajoTestingUtility; +import org.apache.tajo.QueryId; +import org.apache.tajo.QueryIdFactory; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.resource.NodeResource; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.worker.event.ExecutionBlockStartEvent; import org.apache.tajo.worker.event.ExecutionBlockStopEvent; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.apache.tajo.worker.event.TaskStartEvent; import org.junit.After; import org.junit.Before; import org.junit.Test; -import java.util.concurrent.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; import static org.junit.Assert.*; @@ -140,22 +143,12 @@ public void tearDown() { @Test(timeout = 10000) public void testExecutionBlockStart() throws Exception { int requestSize = 1; - - TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); - ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setQueryContext(new QueryContext(conf).getProto()) - .setPlanJson("test") - .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); - CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); requestProto.setExecutionBlockId(ebId.getProto()); - requestProto.setExecutionBlockRequest(ebRequestProto.build()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); @@ -170,18 +163,14 @@ public void testExecutionBlockStart() throws Exception { @Test(timeout = 10000) public void testExecutionBlockStop() throws Exception { - TajoWorkerProtocol.StartExecutionBlockRequestProto.Builder - ebRequestProto = TajoWorkerProtocol.StartExecutionBlockRequestProto.newBuilder(); QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + TaskAllocationRequestProto requestProto = + MockNodeResourceManager.createTaskRequests(ebId, taskMemory, 1).poll(); - ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(workerContext.getConnectionInfo().getProto()) - .setQueryContext(new QueryContext(conf).getProto()) - .setPlanJson("test") - .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + taskDispatcher.getEventHandler().handle(new TaskStartEvent(requestProto.getTaskRequest(), + new NodeResource(requestProto.getResource()))); - taskDispatcher.getEventHandler().handle(new ExecutionBlockStartEvent(ebRequestProto.build())); assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); assertNotNull(taskManager.getExecutionBlockContext(ebId)); assertEquals(ebId, taskManager.getExecutionBlockContext(ebId).getExecutionBlockId()); From 07b7b91f8c3cb8614dcf4ecc6e19d6e790a220ec Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sun, 21 Jun 2015 23:47:44 +0900 Subject: [PATCH 15/80] remove unused class --- .../tajo/worker/event/TaskExecutorEvent.java | 44 ------------------- 1 file changed, 44 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java deleted file mode 100644 index c609c67150..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java +++ /dev/null @@ -1,44 +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.worker.event; - -import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.tajo.TaskAttemptId; - -public class TaskExecutorEvent extends AbstractEvent { - - // producer: NodeResourceManager, consumer: TaskExecutorEvent - public enum EventType { - START, - KILL, - ABORT - } - - private TaskAttemptId taskAttemptId; - - public TaskExecutorEvent(EventType eventType, - TaskAttemptId taskAttemptId) { - super(eventType); - this.taskAttemptId = taskAttemptId; - } - - public TaskAttemptId getTaskId() { - return taskAttemptId; - } -} From ee0c34e59df250e5b88f91b2d8c2d6b231318bde Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 22 Jun 2015 00:07:40 +0900 Subject: [PATCH 16/80] add internal error handling in DefaultTaskScheduler --- .../apache/tajo/querymaster/DefaultTaskScheduler.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 71b087c987..91d92461ae 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -103,6 +103,7 @@ public void run() { schedule(); } catch (Throwable e) { LOG.fatal(e.getMessage(), e); + stage.abort(StageState.ERROR); break; } } @@ -269,7 +270,6 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { request.setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()); request.addAllCandidateNodes(getWorkerIds(getLeafTaskHosts())); masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); - LOG.info("request container:" + remainingScheduledObjectNum()); QueryCoordinatorProtocol.NodeResourceResponseProto responseProto = callBack.get(); for (QueryCoordinatorProtocol.AllocationResourceProto proto : responseProto.getResourceList()) { @@ -777,8 +777,8 @@ public int compare(HostVolumeMapping v1, HostVolumeMapping v2) { rackLocalAssigned++; totalAssigned++; - LOG.info(String.format("Assigned Local/Rack/Total: (%d/%d/%d), Locality: %.2f%%, Rack host: %s", - hostLocalAssigned, rackLocalAssigned, totalAssigned, + LOG.info(String.format("Assigned Local/Rack/Cancel/Total: (%d/%d/%d/%d), Locality: %.2f%%, Rack host: %s", + hostLocalAssigned, rackLocalAssigned, cancellation, totalAssigned, ((double) hostLocalAssigned / (double) totalAssigned) * 100, host)); } @@ -848,8 +848,8 @@ public void assignToLeafTasks(LinkedList taskRequests) { leafTasks.remove(attemptId); rackLocalAssigned++; totalAssigned++; - LOG.info(String.format("Assigned Local/Remote/Total: (%d/%d/%d), Locality: %.2f%%,", - hostLocalAssigned, rackLocalAssigned, totalAssigned, + LOG.info(String.format("Assigned Local/Remote/Cancel/Total: (%d/%d/%d/%d), Locality: %.2f%%,", + hostLocalAssigned, rackLocalAssigned, cancellation, totalAssigned, ((double) hostLocalAssigned / (double) totalAssigned) * 100)); } } From 9b1e779805504acd9cb460b1324f6a4bbc723632 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 22 Jun 2015 07:56:21 +0900 Subject: [PATCH 17/80] fix race condition of FunctionDesc --- .../main/java/org/apache/tajo/catalog/FunctionDesc.java | 7 +------ .../apache/tajo/querymaster/QueryMasterManagerService.java | 4 ++++ .../src/test/java/org/apache/tajo/TajoTestingCluster.java | 2 +- .../src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java | 6 ++---- 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java index 6ea6ac6138..9f71e8e5af 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java @@ -38,7 +38,6 @@ * */ public class FunctionDesc implements ProtoObject, Cloneable, GsonObject, Comparable { - private FunctionDescProto.Builder builder = FunctionDescProto.newBuilder(); @Expose private FunctionSignature signature; @Expose private FunctionInvocation invocation; @@ -184,11 +183,7 @@ public Object clone() throws CloneNotSupportedException{ @Override public FunctionDescProto getProto() { - if (builder == null) { - builder = FunctionDescProto.newBuilder(); - } else { - builder.clear(); - } + FunctionDescProto.Builder builder = FunctionDescProto.newBuilder(); builder.setSignature(signature.getProto()); builder.setSupplement(supplement.getProto()); builder.setInvocation(invocation.getProto()); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index 6594d7c43a..2c8c8eb65d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -191,6 +191,10 @@ public void getExecutionBlockContext(RpcController controller, QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask( new QueryId(request.getExecutionBlockId().getQueryId())); + if(queryMasterTask == null) { + LOG.warn("Query was stopped. request ebId: " + new ExecutionBlockId(request.getExecutionBlockId())); + return; + } Stage stage = queryMasterTask.getQuery().getStage(new ExecutionBlockId(request.getExecutionBlockId())); // first request with starting ExecutionBlock diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index ca663997b0..65a05b1c2a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -123,7 +123,7 @@ void initPropertiesAndConfigs() { conf.setClassVar(ConfVars.GLOBAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, GlobalPlanTestRuleProvider.class); conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 2000); - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 2); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 3); // Client API RPC conf.setIntVar(ConfVars.RPC_CLIENT_WORKER_THREAD_NUM, 2); diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java index 134b3cf537..88ffaf67de 100644 --- a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java +++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java @@ -131,10 +131,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) RemoteCallException callException = (RemoteCallException) cause; ctx.writeAndFlush(callException.getResponse()); - if(LOG.isDebugEnabled()) { - Throwable rootCause = ExceptionUtils.getRootCause(cause); - LOG.error(ExceptionUtils.getMessage(rootCause), rootCause); - } + Throwable rootCause = ExceptionUtils.getRootCause(cause); + LOG.error(ExceptionUtils.getMessage(rootCause), rootCause); } else { /* unhandled exception. */ if (ctx.channel().isOpen()) { From 8b814d08774159e9758a61eee173e6d374594dc3 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 22 Jun 2015 11:14:10 +0900 Subject: [PATCH 18/80] fix unit test failure --- .../apache/tajo/querymaster/DefaultTaskScheduler.java | 11 +++++++++-- .../tajo/querymaster/QueryMasterManagerService.java | 4 ---- .../apache/tajo/engine/query/TestOuterJoinQuery.java | 4 +--- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 91d92461ae..dd8d1530de 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -278,12 +278,19 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { taskRequests.handle(taskRequestEvent); } - if(responseProto.getResourceCount() == 0) { + if(remainingScheduledObjectNum() == 0) { synchronized (schedulingThread){ - schedulingThread.wait(100); + schedulingThread.wait(500); + } + } else { + if(responseProto.getResourceCount() == 0) { + synchronized (schedulingThread){ + schedulingThread.wait(50); + } } } + } catch (Throwable e) { LOG.error(e.getMessage(), e); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index 2c8c8eb65d..6594d7c43a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -191,10 +191,6 @@ public void getExecutionBlockContext(RpcController controller, QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask( new QueryId(request.getExecutionBlockId().getQueryId())); - if(queryMasterTask == null) { - LOG.warn("Query was stopped. request ebId: " + new ExecutionBlockId(request.getExecutionBlockId())); - return; - } Stage stage = queryMasterTask.getQuery().getStage(new ExecutionBlockId(request.getExecutionBlockId())); // first request with starting ExecutionBlock diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java index 944555758f..4d33569600 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestOuterJoinQuery.java @@ -28,8 +28,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.sql.ResultSet; - @Category(IntegrationTest.class) @RunWith(Parameterized.class) @NamedTest("TestJoinQuery") @@ -251,7 +249,7 @@ public final void testLeftOuterJoinPredicationCaseByCase2() throws Exception { } @Test - @Option(withExplain = true, withExplainGlobal = true, parameterized = true) + @Option(withExplain = true, withExplainGlobal = true, parameterized = true, sort = true) @SimpleTest(queries = { @QuerySpec("select t1.id, t1.name, t2.id, t3.id, t4.id\n" + "from jointable11 t1\n" + From 2068b3b98b5c86ad1cc8a24146eb03b9040779c1 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 22 Jun 2015 13:35:47 +0900 Subject: [PATCH 19/80] cleanup and remove duplicated written test case --- .../master/scheduler/SimpleScheduler.java | 2 +- .../org/apache/tajo/querymaster/Query.java | 1 - .../QueryMasterManagerService.java | 34 ++--- .../org/apache/tajo/worker/TaskManager.java | 29 ++-- .../org/apache/tajo/worker/TestHistory.java | 125 ------------------ 5 files changed, 33 insertions(+), 158 deletions(-) delete mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 5797749410..943e5d992a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -301,7 +301,7 @@ public void run() { queryQueue.add(query); synchronized (this) { try { - this.wait(100); + this.wait(1000); } catch (InterruptedException e) { if(!isStopped) { LOG.fatal(e.getMessage(), e); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java index 23808b5583..6e7a8c5a4d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java @@ -303,7 +303,6 @@ private QueryHistory makeQueryHistory() { queryHistory.setQueryId(getId().toString()); queryHistory.setQueryMaster(context.getQueryMasterContext().getWorkerContext().getWorkerName()); queryHistory.setHttpPort(context.getQueryMasterContext().getWorkerContext().getConnectionInfo().getHttpInfoPort()); - queryHistory.setLogicalPlan(plan.toString()); queryHistory.setLogicalPlan(plan.getLogicalPlan().toString()); queryHistory.setDistributedPlan(plan.toString()); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index 6594d7c43a..d96c97deaf 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -117,26 +117,26 @@ public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatus QueryId queryId = new QueryId(request.getId().getTaskId().getExecutionBlockId().getQueryId()); TaskAttemptId attemptId = new TaskAttemptId(request.getId()); QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId); - if (queryMasterTask == null) { - queryMasterTask = queryMaster.getQueryMasterTask(queryId, true); - } - Stage sq = queryMasterTask.getQuery().getStage(attemptId.getTaskId().getExecutionBlockId()); - Task task = sq.getTask(attemptId.getTaskId()); - TaskAttempt attempt = task.getAttempt(attemptId.getId()); - if(LOG.isDebugEnabled()){ - LOG.debug(String.format("Task State: %s, Attempt State: %s", task.getState().name(), attempt.getState().name())); - } + if (queryMasterTask != null) { + Stage sq = queryMasterTask.getQuery().getStage(attemptId.getTaskId().getExecutionBlockId()); + Task task = sq.getTask(attemptId.getTaskId()); + TaskAttempt attempt = task.getAttempt(attemptId.getId()); + + if(LOG.isDebugEnabled()){ + LOG.debug(String.format("Task State: %s, Attempt State: %s", task.getState().name(), attempt.getState().name())); + } + + if (request.getState() == TajoProtos.TaskAttemptState.TA_KILLED) { + LOG.warn(attemptId + " Killed"); + attempt.handle( + new TaskAttemptEvent(new TaskAttemptId(request.getId()), TaskAttemptEventType.TA_LOCAL_KILLED)); + } else { + queryMasterTask.getEventHandler().handle( + new TaskAttemptStatusUpdateEvent(new TaskAttemptId(request.getId()), request)); + } - if (request.getState() == TajoProtos.TaskAttemptState.TA_KILLED) { - LOG.warn(attemptId + " Killed"); - attempt.handle( - new TaskAttemptEvent(new TaskAttemptId(request.getId()), TaskAttemptEventType.TA_LOCAL_KILLED)); - } else { - queryMasterTask.getEventHandler().handle( - new TaskAttemptStatusUpdateEvent(new TaskAttemptId(request.getId()), request)); } - done.run(TajoWorker.NULL_PROTO); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index aff9d4b0ac..d5cf381d82 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -31,21 +31,25 @@ import org.apache.tajo.TaskId; import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.rpc.*; +import org.apache.tajo.rpc.AsyncRpcClient; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.rpc.RpcClientManager; +import org.apache.tajo.rpc.RpcConstants; import org.apache.tajo.util.NetUtils; import org.apache.tajo.util.TUtil; -import org.apache.tajo.worker.event.*; - +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeStatusEvent; +import org.apache.tajo.worker.event.TaskManagerEvent; +import org.apache.tajo.worker.event.TaskStartEvent; import java.io.IOException; import java.net.InetSocketAddress; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockContextRequestProto; import static org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockContextProto; +import static org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockContextRequestProto; /** * A TaskManager is responsible for managing executionBlock resource and tasks. @@ -210,17 +214,14 @@ public Task getTaskByTaskAttemptId(TaskAttemptId taskAttemptId) { return null; } - public List getTaskHistories(ExecutionBlockId executionblockId) throws IOException { - List histories = new ArrayList(); - ExecutionBlockContext context = executionBlockContextMap.get(executionblockId); - if (context != null) { - histories.addAll(context.getTaskHistories().values()); - } - //TODO get List from HistoryReader - return histories; + public List getTaskHistories(ExecutionBlockId executionblockId) + throws IOException { + + return getWorkerContext().getHistoryReader().getTaskHistory(executionblockId.getQueryId().toString(), + executionblockId.toString()); } - public TaskHistory getTaskHistory(TaskId taskId) { + public TaskHistory getTaskHistory(TaskId taskId) throws IOException { TaskHistory history = null; ExecutionBlockContext context = executionBlockContextMap.get(taskId.getExecutionBlockId()); if (context != null) { diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java deleted file mode 100644 index dac8bdce2a..0000000000 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java +++ /dev/null @@ -1,125 +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.worker; - -import com.google.protobuf.ServiceException; -import org.apache.hadoop.service.Service; -import org.apache.tajo.TajoProtos; -import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.benchmark.TPCH; -import org.apache.tajo.client.TajoClient; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.master.QueryInfo; -import org.apache.tajo.master.TajoMaster; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.util.Collection; -import java.util.Map; - -import static org.junit.Assert.*; - -public class TestHistory { - private static TajoTestingCluster cluster; - private static TajoMaster master; - private static TajoConf conf; - private static TajoClient client; - - @BeforeClass - public static void setUp() throws Exception { - cluster = new TajoTestingCluster(); - cluster.startMiniClusterInLocal(1); - master = cluster.getMaster(); - conf = cluster.getConfiguration(); - client = cluster.newTajoClient(); - File file = TPCH.getDataFile("lineitem"); - client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) " - + "using text location 'file://" + file.getAbsolutePath() + "'"); - assertTrue(client.existTable("default.lineitem")); - } - - @AfterClass - public static void tearDown() throws IOException { - if (client != null) client.close(); - if (cluster != null) cluster.shutdownMiniCluster(); - } - - @Test - public final void testTaskRunnerHistory() throws IOException, ServiceException, InterruptedException { - /* int beforeFinishedQueriesCount = master.getContext().getQueryJobManager().getFinishedQueries().size(); - client.executeQueryAndGetResult("select count(*) from lineitem"); - - Collection finishedQueries = master.getContext().getQueryJobManager().getFinishedQueries(); - assertTrue(finishedQueries.size() > beforeFinishedQueriesCount); - - TajoWorker worker = cluster.getTajoWorkers().get(0); - TaskRunnerManager taskRunnerManager = worker.getWorkerContext().getTaskRunnerManager(); - assertNotNull(taskRunnerManager); - - - Collection histories = taskRunnerManager.getExecutionBlockHistories(); - assertTrue(histories.size() > 0); - - TaskRunnerHistory history = histories.iterator().next(); - assertEquals(Service.STATE.STOPPED, history.getState()); - TaskRunnerHistory fromProto = new TaskRunnerHistory(history.getProto()); - assertEquals(history.getExecutionBlockId(), fromProto.getExecutionBlockId()); - assertEquals(history.getFinishTime(), fromProto.getFinishTime()); - assertEquals(history.getStartTime(), fromProto.getStartTime()); - assertEquals(history.getState(), fromProto.getState()); - assertEquals(history.getContainerId(), fromProto.getContainerId()); - assertEquals(history.getProto().getTaskHistoriesCount(), fromProto.getProto().getTaskHistoriesCount()); - */ - } - - @Test - public final void testTaskHistory() throws IOException, ServiceException, InterruptedException { - /*int beforeFinishedQueriesCount = master.getContext().getQueryJobManager().getFinishedQueries().size(); - client.executeQueryAndGetResult("select count(*) from lineitem"); - - Collection finishedQueries = master.getContext().getQueryJobManager().getFinishedQueries(); - assertTrue(finishedQueries.size() > beforeFinishedQueriesCount); - - TajoWorker worker = cluster.getTajoWorkers().get(0); - TaskRunnerManager taskRunnerManager = worker.getWorkerContext().getTaskRunnerManager(); - assertNotNull(taskRunnerManager); - - - Collection histories = taskRunnerManager.getExecutionBlockHistories(); - assertTrue(histories.size() > 0); - - TaskRunnerHistory history = histories.iterator().next(); - - assertTrue(history.size() > 0); - assertEquals(Service.STATE.STOPPED, history.getState()); - - Map.Entry entry = - history.getTaskHistoryMap().entrySet().iterator().next(); - - TaskAttemptId taskAttemptId = entry.getKey(); - TaskHistory taskHistory = entry.getValue(); - - assertEquals(TajoProtos.TaskAttemptState.TA_SUCCEEDED, taskHistory.getState()); - assertEquals(taskAttemptId, taskHistory.getTaskAttemptId());*/ - } -} From 5e6d96a68d75653f3372de650e4a963d2ca4e909 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 22 Jun 2015 14:12:44 +0900 Subject: [PATCH 20/80] change queryMetrics to async --- .../tajo/querymaster/DefaultTaskScheduler.java | 15 +++++++-------- .../apache/tajo/querymaster/QueryMasterTask.java | 7 ++++++- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index dd8d1530de..17ef4c9eae 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -278,19 +278,18 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { taskRequests.handle(taskRequestEvent); } - if(remainingScheduledObjectNum() == 0) { - synchronized (schedulingThread){ - schedulingThread.wait(500); - } - } else { - if(responseProto.getResourceCount() == 0) { + if(responseProto.getResourceCount() == 0) { + if(remainingScheduledObjectNum() == 0) { + // all task assigned complete + synchronized (schedulingThread){ + schedulingThread.wait(500); + } + } else { synchronized (schedulingThread){ schedulingThread.wait(50); } } } - - } catch (Throwable e) { LOG.error(e.getMessage(), e); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index a68d99ac64..8c7b0bf717 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -207,7 +207,12 @@ public void serviceStop() throws Exception { } if (queryMetrics != null) { - queryMetrics.report(new MetricsConsoleReporter()); + queryMasterContext.getEventExecutor().submit(new Runnable() { + @Override + public void run() { + queryMetrics.report(new MetricsConsoleReporter()); + } + }); } super.serviceStop(); From 9dc22e89d90a6a5303ccd0eebfd1a59ff987dd1d Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 6 Jul 2015 16:16:00 +0900 Subject: [PATCH 21/80] add retry count in web ui --- .../src/main/resources/webapps/worker/querytasks.jsp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp index 44c2f805b0..0218e13ea9 100644 --- a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp @@ -221,7 +221,7 @@ %>
# Tasks: <%=numOfTasks%> / # Pages: <%=totalPage%>
- + <% for(Task eachTask : tasks) { int taskSeq = eachTask.getId().getId(); @@ -243,12 +243,13 @@ } %> - + - - - + + + + <% From 9277dd1477fa87f3b6cabf99d4bc83ff408d7463 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 6 Jul 2015 18:13:40 +0900 Subject: [PATCH 22/80] ignore ta_update on ta_unassigned --- .../main/java/org/apache/tajo/querymaster/TaskAttempt.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java index afd3f2a3dc..0faee5e6c0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java @@ -90,6 +90,10 @@ TaskAttemptEventType.TA_RESCHEDULE, new TaskAttemptScheduleTransition()) .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_KILL_WAIT, TaskAttemptEventType.TA_KILL, new KillUnassignedTaskTransition()) + // Ignore-able transitions + .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_UNASSIGNED, + EnumSet.of( + TaskAttemptEventType.TA_UPDATE)) // Transitions from TA_ASSIGNED state .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_ASSIGNED, From b8e12cce618287cc28f22ff8092be907b27d9dbc Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 7 Jul 2015 12:35:00 +0900 Subject: [PATCH 23/80] fix resource leak for killing query --- .../querymaster/AbstractTaskScheduler.java | 6 ++-- .../querymaster/DefaultTaskScheduler.java | 23 +++++++++--- .../org/apache/tajo/querymaster/Query.java | 7 +++- .../QueryMasterManagerService.java | 35 ++++++++++--------- .../org/apache/tajo/querymaster/Stage.java | 5 +-- .../org/apache/tajo/worker/TaskExecutor.java | 16 ++++++--- .../org/apache/tajo/worker/TaskManager.java | 24 +++++++------ 7 files changed, 76 insertions(+), 40 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java index b694d5e840..53657d4337 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java @@ -33,7 +33,7 @@ public abstract class AbstractTaskScheduler extends AbstractService implements E protected int rackLocalAssigned; protected int totalAssigned; protected int cancellation; - protected Set leafTaskHosts = Sets.newHashSet(); + protected Set assignedHosts = Sets.newHashSet(); /** * Construct the service. @@ -61,7 +61,7 @@ public int getTotalAssigned() { public abstract int remainingScheduledObjectNum(); - public Set getLeafTaskHosts(){ - return leafTaskHosts; + public Set getAssignedWorker(){ + return assignedHosts; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 17ef4c9eae..50a786d69f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -74,6 +74,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler { private int nextTaskId = 0; private int scheduledObjectNum = 0; + boolean isLeaf; public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { super(DefaultTaskScheduler.class.getName()); @@ -85,7 +86,18 @@ public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { public void init(Configuration conf) { scheduledRequests = new ScheduledRequests(); - taskRequests = new TaskRequests(); + taskRequests = new TaskRequests(); + + isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); + if (!isLeaf) { + + //find assigned hosts for interQuery locality in children executionBlock + List executionBlockList = stage.getMasterPlan().getChilds(stage.getBlock()); + for (ExecutionBlock executionBlock : executionBlockList) { + Stage childStage = stage.getContext().getStage(executionBlock.getId()); + assignedHosts.addAll(childStage.getTaskScheduler().getAssignedWorker()); + } + } super.init(conf); } @@ -268,7 +280,7 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { QueryCoordinatorProtocol.ResourceType.INTERMEDIATE); request.setUserId(context.getMasterContext().getQueryContext().getUser()); request.setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()); - request.addAllCandidateNodes(getWorkerIds(getLeafTaskHosts())); + request.addAllCandidateNodes(getWorkerIds(getAssignedWorker())); masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); QueryCoordinatorProtocol.NodeResourceResponseProto responseProto = callBack.get(); @@ -311,7 +323,6 @@ public void handle(TaskRequestEvent event) { } if(stopEventHandling.get()) { - //event.getCallback().run(stopTaskRunnerReq); return; } int qSize = taskRequestQueue.size(); @@ -655,7 +666,7 @@ private synchronized void addLeafTask(TaskAttemptToSchedulerEvent event) { for (DataLocation location : locations) { String host = location.getHost(); - leafTaskHosts.add(host); + assignedHosts.add(host); HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); if (hostVolumeMapping == null) { @@ -913,6 +924,8 @@ public void assignToLeafTasks(LinkedList taskRequests) { } context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); + //TODO change to debug + LOG.info("Assigned task: " + attemptId + " to " + connectionInfo.getHostAndPeerRpcPort()); assignedRequest.add(attemptId); scheduledObjectNum--; @@ -1014,6 +1027,8 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { } context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); + //TODO change to debug + LOG.info("Assigned task: " + attemptId + " to " + connectionInfo.getHostAndPeerRpcPort()); totalAssigned++; scheduledObjectNum--; } catch (Exception e) { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java index 54c059b81d..9ade319384 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java @@ -174,7 +174,12 @@ public class Query implements EventHandler { QueryEventType.KILL, QUERY_COMPLETED_TRANSITION) - // Transitions from FAILED state + // Transitions from KILLED state + // ignore-able transitions + .addTransition(QueryState.QUERY_KILLED, QueryState.QUERY_KILLED, + EnumSet.of(QueryEventType.QUERY_COMPLETED, QueryEventType.KILL)) + + // Transitions from FAILED state .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED, QueryEventType.DIAGNOSTIC_UPDATE, DIAGNOSTIC_UPDATE_TRANSITION) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index d96c97deaf..d2a43f32a6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -190,27 +190,30 @@ public void getExecutionBlockContext(RpcController controller, QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask( new QueryId(request.getExecutionBlockId().getQueryId())); + if (queryMasterTask != null) { - Stage stage = queryMasterTask.getQuery().getStage(new ExecutionBlockId(request.getExecutionBlockId())); + Stage stage = queryMasterTask.getQuery().getStage(new ExecutionBlockId(request.getExecutionBlockId())); - // first request with starting ExecutionBlock - PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); + // first request with starting ExecutionBlock + PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); - TajoWorkerProtocol.ExecutionBlockContextProto.Builder - ebRequestProto = TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); - ebRequestProto.setExecutionBlockId(request.getExecutionBlockId()) - .setQueryContext(stage.getContext().getQueryContext().getProto()) - .setQueryOutputPath(stage.getContext().getStagingDir().toString()) - .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) - .setShuffleType(shuffleType); + TajoWorkerProtocol.ExecutionBlockContextProto.Builder + ebRequestProto = TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); + ebRequestProto.setExecutionBlockId(request.getExecutionBlockId()) + .setQueryContext(stage.getContext().getQueryContext().getProto()) + .setQueryOutputPath(stage.getContext().getStagingDir().toString()) + .setPlanJson(CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class)) + .setShuffleType(shuffleType); - //Set assigned worker to stage - if (!stage.getWorkerMap().containsKey(request.getWorker().getId())) { - stage.getWorkerMap().put(request.getWorker().getId(), - NetUtils.createSocketAddr(request.getWorker().getHost(), request.getWorker().getPeerRpcPort())); + //Set assigned worker to stage + if (!stage.getWorkerMap().containsKey(request.getWorker().getId())) { + stage.getWorkerMap().put(request.getWorker().getId(), + NetUtils.createSocketAddr(request.getWorker().getHost(), request.getWorker().getPeerRpcPort())); + } + done.run(ebRequestProto.build()); + } else { + controller.setFailed("Can't find query. request: " + request); } - - done.run(ebRequestProto.build()); } @Override diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 9d0021da22..20af0a6362 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -267,7 +267,8 @@ StageEventType.SQ_KILL, new KillTasksTransition()) StageEventType.SQ_KILL, StageEventType.SQ_FAILED, StageEventType.SQ_INTERNAL_ERROR, - StageEventType.SQ_STAGE_COMPLETED)) + StageEventType.SQ_STAGE_COMPLETED, + StageEventType.SQ_SHUFFLE_REPORT)) .installTopology(); @@ -674,7 +675,7 @@ private void stopScheduler() { } /** - * Get the task launched worker + * Get the launched worker address */ protected Map getWorkerMap() { return workerMap; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index c4f243201c..90bb6bbcd2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -123,11 +123,19 @@ protected Task getNextTask() { return task; } - @SuppressWarnings("unchecked") protected void stopTask(TaskAttemptId taskId) { runningTasks.decrementAndGet(); - workerContext.getNodeResourceManager().getDispatcher().getEventHandler().handle( - new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId), NodeResourceEvent.ResourceType.TASK)); + releaseResource(taskId); + } + + @SuppressWarnings("unchecked") + protected void releaseResource(TaskAttemptId taskId) { + NodeResource resource = allocatedResourceMap.remove(taskId); + + if(resource != null) { + workerContext.getNodeResourceManager().getDispatcher().getEventHandler().handle( + new NodeResourceDeallocateEvent(resource, NodeResourceEvent.ResourceType.TASK)); + } } protected ExecutorService getFetcherExecutor() { @@ -177,7 +185,7 @@ public void handle(TaskStartEvent event) { if (!isStopped) { LOG.fatal(e.getMessage(), e); } - } catch (IOException e) { + } catch (Exception e) { stopTask(event.getTaskAttemptId()); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index d5cf381d82..fd5f4edb45 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -177,17 +177,21 @@ public void handle(TaskManagerEvent event) { case TASK_START: { //receive event from NodeResourceManager TaskStartEvent taskStartEvent = TUtil.checkTypeAndGet(event, TaskStartEvent.class); - if (!executionBlockContextMap.containsKey(taskStartEvent.getExecutionBlockId())) { - ExecutionBlockContext context = createExecutionBlock(taskStartEvent.getExecutionBlockId(), - taskStartEvent.getTaskRequest().getQueryMasterHostAndPort()); - - executionBlockContextMap.put(context.getExecutionBlockId(), context); - LOG.info("Running ExecutionBlocks: " + executionBlockContextMap.size() - + ", running tasks:" + getRunningTasks() + ", availableResource: " - + workerContext.getNodeResourceManager().getAvailableResource()); + try { + if (!executionBlockContextMap.containsKey(taskStartEvent.getExecutionBlockId())) { + ExecutionBlockContext context = createExecutionBlock(taskStartEvent.getExecutionBlockId(), + taskStartEvent.getTaskRequest().getQueryMasterHostAndPort()); + + executionBlockContextMap.put(context.getExecutionBlockId(), context); + LOG.info("Running ExecutionBlocks: " + executionBlockContextMap.size() + + ", running tasks:" + getRunningTasks() + ", availableResource: " + + workerContext.getNodeResourceManager().getAvailableResource()); + } + getTaskExecutor().handle(taskStartEvent); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + getTaskExecutor().releaseResource(taskStartEvent.getTaskAttemptId()); } - - getTaskExecutor().handle(taskStartEvent); break; } case EB_STOP: { From 386a2ae15080b4ab3e8103a7654da7ded5964c32 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 7 Jul 2015 14:01:02 +0900 Subject: [PATCH 24/80] fix history race condition --- .../main/java/org/apache/tajo/worker/ExecutionBlockContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 2f457e8424..03a8958904 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -91,7 +91,7 @@ public class ExecutionBlockContext { // It keeps all of the query unit attempts while a TaskRunner is running. private final ConcurrentMap tasks = Maps.newConcurrentMap(); - private final Map taskHistories = Maps.newTreeMap(); + private final Map taskHistories = Maps.newConcurrentMap(); public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, ExecutionBlockContextProto request, AsyncRpcClient queryMasterClient) From c2579c8cb8ffd50dc4c9071d80417ecff88d014b Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 7 Jul 2015 20:58:19 +0900 Subject: [PATCH 25/80] fix add cancellation event in TaskAttempt and fix query kill bug of multiful stage --- .../master/event/TaskAttemptEventType.java | 1 + .../querymaster/AbstractTaskScheduler.java | 6 ++- .../querymaster/DefaultTaskScheduler.java | 42 +++++++++---------- .../org/apache/tajo/querymaster/Query.java | 13 +++++- .../tajo/querymaster/QueryMasterTask.java | 19 ++++----- .../org/apache/tajo/querymaster/Stage.java | 6 +-- .../org/apache/tajo/querymaster/Task.java | 4 -- .../apache/tajo/querymaster/TaskAttempt.java | 16 +++++-- .../org/apache/tajo/worker/TaskContainer.java | 1 - 9 files changed, 60 insertions(+), 48 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java index e35b1543f1..f59b50b996 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptEventType.java @@ -33,6 +33,7 @@ public enum TaskAttemptEventType { //Producer:Scheduler TA_ASSIGNED, + TA_ASSIGN_CANCEL, TA_SCHEDULE_CANCELED, //Producer:Scheduler diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java index 53657d4337..db1439def6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java @@ -21,7 +21,6 @@ import com.google.common.collect.Sets; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tajo.master.event.TaskRequestEvent; import org.apache.tajo.master.event.TaskSchedulerEvent; import java.util.Set; @@ -56,7 +55,10 @@ public int getTotalAssigned() { return totalAssigned; } - public abstract void handleTaskRequestEvent(TaskRequestEvent event); + public int getCancellation() { + return cancellation; + } + public abstract void releaseTaskAttempt(TaskAttempt taskAttempt); public abstract int remainingScheduledObjectNum(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 50a786d69f..c3e490206f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -148,7 +148,7 @@ public void stop() { LinkedList taskRequestEvents = new LinkedList(); public void schedule() { - handleTaskRequestEvent(null); + reserveNodeResource(); if (taskRequests.size() > 0) { if (scheduledRequests.leafTaskNum() > 0) { @@ -254,8 +254,8 @@ private List getWorkerIds(Collection hosts){ return workerIds; } - @Override - public void handleTaskRequestEvent(TaskRequestEvent event) { + + private void reserveNodeResource() { boolean isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); int taskMem = context.getMasterContext().getConf().getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); @@ -292,7 +292,7 @@ public void handleTaskRequestEvent(TaskRequestEvent event) { if(responseProto.getResourceCount() == 0) { if(remainingScheduledObjectNum() == 0) { - // all task assigned complete + // all task is assigned, wait for stopping message synchronized (schedulingThread){ schedulingThread.wait(500); } @@ -649,6 +649,9 @@ public void cancel(TaskAttempt taskAttempt) { } else { scheduledRequests.nonLeafTasks.add(taskAttempt.getId()); } + + context.getMasterContext().getEventHandler().handle( + new TaskAttemptEvent(taskAttempt.getId(), TaskAttemptEventType.TA_ASSIGN_CANCEL)); } private class ScheduledRequests { @@ -818,10 +821,7 @@ public void assignToLeafTasks(LinkedList taskRequests) { // checking if this container is still alive. // If not, ignore the task request and stop the task runner WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); - if(connectionInfo == null) { - //taskRequest.getCallback().run(stopTaskRunnerReq); - continue; - } + if(connectionInfo == null) continue; // getting the hostname of requested node String host = connectionInfo.getHost(); @@ -872,7 +872,6 @@ public void assignToLeafTasks(LinkedList taskRequests) { } } - if (attemptId != null) { Task task = stage.getTask(attemptId.getTaskId()); TaskRequest taskAssign = new TaskRequestImpl( @@ -897,35 +896,34 @@ public void assignToLeafTasks(LinkedList taskRequests) { .setTaskRequest(taskAssign.getProto()).build()); requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); - - CallFuture callFuture = new CallFuture(); + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); InetSocketAddress addr = stage.getWorkerMap().get(connectionInfo.getId()); if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); AsyncRpcClient tajoWorkerRpc = null; + CallFuture callFuture = new CallFuture(); try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); - TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); - if(responseProto.getCancellationTaskCount() > 0) { + TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); + if (responseProto.getCancellationTaskCount() > 0) { for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); cancellation++; } LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); continue; + } else { + context.getMasterContext().getEventHandler().handle( + new TaskTAttemptEvent(attemptId, + TaskEventType.T_ATTEMPT_LAUNCHED)); } - } catch (Exception e) { LOG.error(e); } - - context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); - //TODO change to debug - LOG.info("Assigned task: " + attemptId + " to " + connectionInfo.getHostAndPeerRpcPort()); assignedRequest.add(attemptId); scheduledObjectNum--; @@ -1003,6 +1001,7 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { .setTaskRequest(taskAssign.getProto()).build()); requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); + context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); CallFuture callFuture = new CallFuture(); @@ -1024,11 +1023,12 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { } LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); continue; + } else { + context.getMasterContext().getEventHandler().handle( + new TaskTAttemptEvent(attemptId, + TaskEventType.T_ATTEMPT_LAUNCHED)); } - context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); - //TODO change to debug - LOG.info("Assigned task: " + attemptId + " to " + connectionInfo.getHostAndPeerRpcPort()); totalAssigned++; scheduledObjectNum--; } catch (Exception e) { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java index 9ade319384..16c4a1f255 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java @@ -708,8 +708,17 @@ public void transition(Query query, QueryEvent event) { !executeNextBlock(query, castEvent.getExecutionBlockId())) { return; } - // if a query is completed due to finished, kill, failure, or error - query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState())); + + //wait for stages is completed + if (query.completedStagesCount == query.stages.size()) { + // if a query is completed due to finished, kill, failure, or error + query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState())); + } + LOG.info(String.format("Complete Stage[%s], State: %s, %d/%d. ", + castEvent.getExecutionBlockId().toString(), + castEvent.getState().toString(), + query.completedStagesCount, + query.stages.size())); } catch (Throwable t) { LOG.error(t.getMessage(), t); query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR)); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index d76156cd6f..8247b9fec5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -49,10 +49,8 @@ import org.apache.tajo.plan.logical.ScanNode; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.resource.NodeResource; -import org.apache.tajo.rpc.AsyncRpcClient; -import org.apache.tajo.rpc.NettyClientBase; -import org.apache.tajo.rpc.NullCallback; -import org.apache.tajo.rpc.RpcClientManager; +import org.apache.tajo.rpc.*; +import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.session.Session; import org.apache.tajo.storage.FormatProperty; import org.apache.tajo.storage.Tablespace; @@ -205,11 +203,6 @@ public void run() { super.serviceStop(); LOG.info("Stopped QueryMasterTask:" + queryId); } - //FIXME remove - public void handleTaskRequestEvent(TaskRequestEvent event) { - ExecutionBlockId id = event.getExecutionBlockId(); - query.getStage(id).handleTaskRequestEvent(event); - } public void handleTaskFailed(TajoWorkerProtocol.TaskFatalErrorReport report) { synchronized(diagnostics) { @@ -283,7 +276,13 @@ protected void killTaskAttempt(int workerId, TaskAttemptId taskAttemptId) { try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(workerAddress, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); - tajoWorkerRpcClient.killTaskAttempt(null, taskAttemptId.getProto(), NullCallback.get()); + CallFuture callFuture = new CallFuture(); + tajoWorkerRpcClient.killTaskAttempt(null, taskAttemptId.getProto(), callFuture); + + if(!callFuture.get().getValue()){ + queryMasterContext.getEventHandler().handle( + new TaskFatalErrorEvent(taskAttemptId, "Can't kill task :" + taskAttemptId)); + } } catch (Exception e) { /* Worker RPC failure */ LOG.error(e.getMessage(), e); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 20af0a6362..5601fdf543 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -789,10 +789,6 @@ public void handle(StageEvent event) { } } - public void handleTaskRequestEvent(TaskRequestEvent event) { - taskScheduler.handleTaskRequestEvent(event); - } - private static class InitAndRequestContainer implements MultipleArcTransition { @@ -1166,7 +1162,7 @@ public void transition(Stage stage, stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_FAILED)); } else { stage.completedTaskCount++; - stage.getTaskScheduler().releaseTaskAttempt(task.getLastAttempt()); //FIXME + stage.getTaskScheduler().releaseTaskAttempt(task.getLastAttempt()); if (taskEvent.getState() == TaskState.SUCCEEDED) { stage.succeededObjectCount++; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java index 0d8a30ce55..c23ba8718d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java @@ -46,7 +46,6 @@ import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.storage.fragment.FragmentConvertor; import org.apache.tajo.util.Pair; -import org.apache.tajo.util.NumberUtil; import org.apache.tajo.util.TajoIdUtils; import org.apache.tajo.util.history.TaskHistory; import org.apache.tajo.worker.FetchImpl; @@ -627,10 +626,7 @@ public void transition(Task task, if (!(event instanceof TaskTAttemptEvent)) { throw new IllegalArgumentException("event should be a TaskTAttemptEvent type."); } - TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event; - TaskAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId()); task.launchTime = System.currentTimeMillis(); - task.succeededWorker = attempt.getWorkerConnectionInfo(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java index 0faee5e6c0..2ed139db7a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java @@ -111,6 +111,8 @@ TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition()) TaskAttemptEventType.TA_DONE, new SucceededTransition()) .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_FAILED, TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition()) + .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_UNASSIGNED, + TaskAttemptEventType.TA_ASSIGN_CANCEL, new CancelTransition()) // Transitions from TA_RUNNING state .addTransition(TaskAttemptState.TA_RUNNING, @@ -310,9 +312,17 @@ public void transition(TaskAttempt taskAttempt, } TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event; taskAttempt.workerConnectionInfo = castEvent.getWorkerConnectionInfo(); - taskAttempt.eventHandler.handle( - new TaskTAttemptEvent(taskAttempt.getId(), - TaskEventType.T_ATTEMPT_LAUNCHED)); + } + } + + private static class CancelTransition + implements SingleArcTransition { + + @Override + public void transition(TaskAttempt taskAttempt, + TaskAttemptEvent event) { + + taskAttempt.workerConnectionInfo = null; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java index 2576726840..761bf5211b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java @@ -20,7 +20,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.tajo.TajoProtos; /** * The driver class for Tajo Task processing. From 594828ce17d3efe8c5ebde8423ff9762aee86a56 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 7 Jul 2015 23:11:51 +0900 Subject: [PATCH 26/80] fix wrong rpc handling and add failure eb cleanup --- .../org/apache/tajo/querymaster/Task.java | 1 + .../tajo/worker/ExecutionBlockContext.java | 3 +- .../tajo/worker/TajoWorkerManagerService.java | 4 ++- .../org/apache/tajo/worker/TaskManager.java | 21 ++++++++--- .../tajo/worker/event/QueryStopEvent.java | 35 +++++++++++++++++++ .../tajo/worker/event/TaskManagerEvent.java | 4 ++- 6 files changed, 60 insertions(+), 8 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/QueryStopEvent.java diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java index c23ba8718d..a396ad0451 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java @@ -174,6 +174,7 @@ public class Task implements EventHandler { EnumSet.of( TaskEventType.T_KILL, TaskEventType.T_SCHEDULE, + TaskEventType.T_ATTEMPT_LAUNCHED, TaskEventType.T_ATTEMPT_SUCCEEDED, TaskEventType.T_ATTEMPT_FAILED)) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 03a8958904..cd8777518d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -76,7 +76,6 @@ public class ExecutionBlockContext { private TajoQueryEngine queryEngine; private RpcClientManager connManager; private AsyncRpcClient queryMasterClient; - private NettyClientBase client; private QueryMasterProtocol.QueryMasterProtocolService.Interface stub; private TajoConf systemConf; // for the doAs block @@ -185,7 +184,7 @@ public void stop(){ tasks.clear(); taskHistories.clear(); resource.release(); - RpcClientManager.cleanup(client); + RpcClientManager.cleanup(queryMasterClient); } public TajoConf getConf() { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index 1910c1b5e6..cf703c370e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -35,6 +35,7 @@ import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.event.ExecutionBlockStopEvent; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.apache.tajo.worker.event.QueryStopEvent; import java.net.InetSocketAddress; @@ -140,7 +141,8 @@ public void killTaskAttempt(RpcController controller, TajoIdProtos.TaskAttemptId @Override public void cleanup(RpcController controller, TajoIdProtos.QueryIdProto request, RpcCallback done) { - workerContext.cleanup(new QueryId(request).toString()); + + workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new QueryStopEvent(new QueryId(request))); done.run(TajoWorker.TRUE_PROTO); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index fd5f4edb45..ef6df940a2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -37,10 +37,7 @@ import org.apache.tajo.rpc.RpcConstants; import org.apache.tajo.util.NetUtils; import org.apache.tajo.util.TUtil; -import org.apache.tajo.worker.event.ExecutionBlockStopEvent; -import org.apache.tajo.worker.event.NodeStatusEvent; -import org.apache.tajo.worker.event.TaskManagerEvent; -import org.apache.tajo.worker.event.TaskStartEvent; +import org.apache.tajo.worker.event.*; import java.io.IOException; import java.net.InetSocketAddress; @@ -203,6 +200,22 @@ public void handle(TaskManagerEvent event) { ((ExecutionBlockStopEvent) event).getCleanupList()); break; } + case QUERY_STOP: { + QueryStopEvent queryStopEvent = TUtil.checkTypeAndGet(event, QueryStopEvent.class); + + //cleanup failure ExecutionBlock + for (ExecutionBlockId ebId : executionBlockContextMap.keySet()) { + if (ebId.getQueryId().equals(queryStopEvent.getQueryId())) { + try { + executionBlockContextMap.remove(ebId).stop(); + } catch (Exception e) { + LOG.fatal(e.getMessage(), e); + } + } + } + workerContext.cleanup(queryStopEvent.getQueryId().toString()); + break; + } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/QueryStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/QueryStopEvent.java new file mode 100644 index 0000000000..892db92e5b --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/QueryStopEvent.java @@ -0,0 +1,35 @@ +/** + * 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.event; + +import org.apache.tajo.QueryId; + +public class QueryStopEvent extends TaskManagerEvent { + + + private QueryId queryId; + public QueryStopEvent(QueryId queryId) { + super(EventType.QUERY_STOP); + this.queryId = queryId; + } + + public QueryId getQueryId() { + return queryId; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java index efbaaf95c9..e1a945cc81 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java @@ -27,7 +27,9 @@ public enum EventType { TASK_KILL, TASK_ABORT, - EB_STOP + //cleanup events + EB_STOP, + QUERY_STOP } From 5907d32a55b9b88e14df672a7af8935cef141282 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 8 Jul 2015 00:28:53 +0900 Subject: [PATCH 27/80] fix invalid event --- .../apache/tajo/querymaster/DefaultTaskScheduler.java | 9 --------- .../java/org/apache/tajo/querymaster/TaskAttempt.java | 7 +++---- 2 files changed, 3 insertions(+), 13 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index c3e490206f..234a565b52 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -916,10 +916,6 @@ public void assignToLeafTasks(LinkedList taskRequests) { } LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); continue; - } else { - context.getMasterContext().getEventHandler().handle( - new TaskTAttemptEvent(attemptId, - TaskEventType.T_ATTEMPT_LAUNCHED)); } } catch (Exception e) { LOG.error(e); @@ -1023,10 +1019,6 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { } LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); continue; - } else { - context.getMasterContext().getEventHandler().handle( - new TaskTAttemptEvent(attemptId, - TaskEventType.T_ATTEMPT_LAUNCHED)); } totalAssigned++; @@ -1034,7 +1026,6 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { } catch (Exception e) { LOG.error(e); } - } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java index 2ed139db7a..b650089632 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java @@ -90,10 +90,6 @@ TaskAttemptEventType.TA_RESCHEDULE, new TaskAttemptScheduleTransition()) .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_KILL_WAIT, TaskAttemptEventType.TA_KILL, new KillUnassignedTaskTransition()) - // Ignore-able transitions - .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_UNASSIGNED, - EnumSet.of( - TaskAttemptEventType.TA_UPDATE)) // Transitions from TA_ASSIGNED state .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_ASSIGNED, @@ -312,6 +308,9 @@ public void transition(TaskAttempt taskAttempt, } TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event; taskAttempt.workerConnectionInfo = castEvent.getWorkerConnectionInfo(); + taskAttempt.eventHandler.handle( + new TaskTAttemptEvent(taskAttempt.getId(), + TaskEventType.T_ATTEMPT_LAUNCHED)); } } From fa0b26c6b250425397720cfb0e46988bc61d4dbd Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 8 Jul 2015 10:04:04 +0900 Subject: [PATCH 28/80] fix broken web ui and remove possible memory leak --- .../java/org/apache/tajo/conf/TajoConf.java | 2 +- .../master/scheduler/SimpleScheduler.java | 5 +- .../querymaster/AbstractTaskScheduler.java | 6 +- .../querymaster/DefaultTaskScheduler.java | 74 +++++++++---------- .../apache/tajo/querymaster/QueryMaster.java | 29 ++++---- .../QueryMasterManagerService.java | 4 +- .../tajo/querymaster/QueryMasterTask.java | 6 +- .../org/apache/tajo/querymaster/Stage.java | 6 +- .../tajo/worker/TajoWorkerClientService.java | 2 +- .../tajo/worker/TajoWorkerManagerService.java | 2 +- .../src/main/proto/TajoWorkerProtocol.proto | 2 +- .../main/resources/webapps/admin/query.jsp | 4 +- .../resources/webapps/worker/querydetail.jsp | 2 +- .../resources/webapps/worker/queryplan.jsp | 2 +- .../resources/webapps/worker/querytasks.jsp | 21 ++---- .../main/resources/webapps/worker/task.jsp | 2 +- 16 files changed, 82 insertions(+), 87 deletions(-) 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 4486dfc2d3..3b8df2983f 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 @@ -185,7 +185,7 @@ public static enum ConfVars implements ConfigKey { // Tajo History WORKER_HISTORY_EXPIRE_PERIOD("tajo.worker.history.expire-interval-minutes", 60), // 1 hours - QUERYMASTER_HISTORY_EXPIRE_PERIOD("tajo.qm.history.expire-interval-minutes", 6 * 60), // 6 hours + QUERYMASTER_CACHE_EXPIRE_PERIOD("tajo.qm.history.expire-interval-minutes", 10), // 10 mins WORKER_HEARTBEAT_INTERVAL("tajo.worker.heartbeat.interval", 10 * 1000), // 10 sec diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 943e5d992a..7b0321874f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -159,7 +159,7 @@ public int getNumClusterNodes() { } int requiredContainers = request.getNumContainers(); - // reserve resource to the candidate workers for locality + // reserve resource from candidate workers for locality reservedResources = reserveClusterResource(workers, capacity, requiredContainers); // reserve resource in random workers @@ -229,6 +229,9 @@ public void handle(SchedulerEvent event) { case RESOURCE_UPDATE: updateResource(); break; + default: + break; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java index db1439def6..8636eaaed8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java @@ -32,7 +32,7 @@ public abstract class AbstractTaskScheduler extends AbstractService implements E protected int rackLocalAssigned; protected int totalAssigned; protected int cancellation; - protected Set assignedHosts = Sets.newHashSet(); + protected Set leafTaskHosts = Sets.newHashSet(); /** * Construct the service. @@ -63,7 +63,7 @@ public int getCancellation() { public abstract int remainingScheduledObjectNum(); - public Set getAssignedWorker(){ - return assignedHosts; + public Set getLeafTaskHosts(){ + return leafTaskHosts; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 234a565b52..02af7f0a24 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -55,7 +56,6 @@ import java.util.*; import java.util.Map.Entry; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; @@ -67,7 +67,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler { private Stage stage; private Thread schedulingThread; - private AtomicBoolean stopEventHandling = new AtomicBoolean(false); + private volatile boolean isStopped; private ScheduledRequests scheduledRequests; private TaskRequests taskRequests; @@ -75,6 +75,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler { private int nextTaskId = 0; private int scheduledObjectNum = 0; boolean isLeaf; + private Set candidateWorkers = Sets.newHashSet(); public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { super(DefaultTaskScheduler.class.getName()); @@ -84,32 +85,32 @@ public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { @Override public void init(Configuration conf) { - scheduledRequests = new ScheduledRequests(); taskRequests = new TaskRequests(); + super.init(conf); + } + + @Override + public void start() { + LOG.info("Start TaskScheduler"); isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); - if (!isLeaf) { - //find assigned hosts for interQuery locality in children executionBlock + if (!isLeaf) { + candidateWorkers.addAll(getWorkerIds(getLeafTaskHosts())); + } else { + //find assigned hosts for Non-Leaf locality in children executionBlock List executionBlockList = stage.getMasterPlan().getChilds(stage.getBlock()); for (ExecutionBlock executionBlock : executionBlockList) { Stage childStage = stage.getContext().getStage(executionBlock.getId()); - assignedHosts.addAll(childStage.getTaskScheduler().getAssignedWorker()); + candidateWorkers.addAll(childStage.getAssignedWorkerMap().keySet()); } } - super.init(conf); - } - - @Override - public void start() { - LOG.info("Start TaskScheduler"); - this.schedulingThread = new Thread() { public void run() { - while (!stopEventHandling.get() && !Thread.currentThread().isInterrupted()) { + while (!isStopped && !Thread.currentThread().isInterrupted()) { try { schedule(); @@ -129,9 +130,7 @@ public void run() { @Override public void stop() { - if(stopEventHandling.getAndSet(true)){ - return; - } + isStopped = true; if (schedulingThread != null) { synchronized (schedulingThread) { @@ -242,8 +241,8 @@ public void handle(TaskSchedulerEvent event) { } } - private List getWorkerIds(Collection hosts){ - List workerIds = Lists.newArrayList(); + private Set getWorkerIds(Collection hosts){ + Set workerIds = Sets.newHashSet(); if(hosts.isEmpty()) return workerIds; for (WorkerConnectionInfo worker : stage.getContext().getWorkerMap().values()) { @@ -257,9 +256,8 @@ private List getWorkerIds(Collection hosts){ private void reserveNodeResource() { - boolean isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); int taskMem = context.getMasterContext().getConf().getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); - NettyClientBase tmClient = null; + NettyClientBase tmClient; try { ServiceTracker serviceTracker = context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); @@ -270,17 +268,17 @@ private void reserveNodeResource() { CallFuture callBack = new CallFuture(); QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); - request.setCapacity(NodeResources.createResource(taskMem, isLeaf ? 1 : 0).getProto()); - request.setNumContainers(Math.max(remainingScheduledObjectNum(), 1)); - request.setPriority(stage.getPriority()); - request.setQueryId(context.getMasterContext().getQueryId().getProto()); - //TODO set queue - request.setQueue(context.getMasterContext().getQueryContext().get("queue", "default")); - request.setType(isLeaf ? QueryCoordinatorProtocol.ResourceType.LEAF: - QueryCoordinatorProtocol.ResourceType.INTERMEDIATE); - request.setUserId(context.getMasterContext().getQueryContext().getUser()); - request.setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()); - request.addAllCandidateNodes(getWorkerIds(getAssignedWorker())); + request.setCapacity(NodeResources.createResource(taskMem, isLeaf ? 1 : 0).getProto()) + .setNumContainers(Math.max(remainingScheduledObjectNum(), 1)) + .setPriority(stage.getPriority()) + .setQueryId(context.getMasterContext().getQueryId().getProto()) + .setType(isLeaf ? QueryCoordinatorProtocol.ResourceType.LEAF : + QueryCoordinatorProtocol.ResourceType.INTERMEDIATE) + .setUserId(context.getMasterContext().getQueryContext().getUser()) + .setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()) + .addAllCandidateNodes(candidateWorkers) + .setQueue(context.getMasterContext().getQueryContext().get("queue", "default")); //TODO set queue + masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); QueryCoordinatorProtocol.NodeResourceResponseProto responseProto = callBack.get(); @@ -322,7 +320,7 @@ public void handle(TaskRequestEvent event) { LOG.debug("TaskRequest: " + event.getResponseProto().getWorkerId() + "," + event.getExecutionBlockId()); } - if(stopEventHandling.get()) { + if(isStopped) { return; } int qSize = taskRequestQueue.size(); @@ -663,13 +661,13 @@ private class ScheduledRequests { private Map leafTaskHostMapping = Maps.newConcurrentMap(); private final Map> leafTasksRackMapping = Maps.newConcurrentMap(); - private synchronized void addLeafTask(TaskAttemptToSchedulerEvent event) { + private void addLeafTask(TaskAttemptToSchedulerEvent event) { TaskAttempt taskAttempt = event.getTaskAttempt(); List locations = taskAttempt.getTask().getDataLocations(); for (DataLocation location : locations) { String host = location.getHost(); - assignedHosts.add(host); + leafTaskHosts.add(host); HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); if (hostVolumeMapping == null) { @@ -898,7 +896,7 @@ public void assignToLeafTasks(LinkedList taskRequests) { requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); - InetSocketAddress addr = stage.getWorkerMap().get(connectionInfo.getId()); + InetSocketAddress addr = stage.getAssignedWorkerMap().get(connectionInfo.getId()); if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); AsyncRpcClient tajoWorkerRpc = null; @@ -1001,10 +999,10 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { CallFuture callFuture = new CallFuture(); - InetSocketAddress addr = stage.getWorkerMap().get(connectionInfo.getId()); + InetSocketAddress addr = stage.getAssignedWorkerMap().get(connectionInfo.getId()); if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); - AsyncRpcClient tajoWorkerRpc = null; + AsyncRpcClient tajoWorkerRpc; try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index d1c6595191..b5d49cd169 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -19,6 +19,7 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Maps; +import org.apache.commons.collections.map.LRUMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -44,6 +45,7 @@ import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.util.TUtil; +import org.apache.tajo.util.history.HistoryReader; import org.apache.tajo.util.history.QueryHistory; import org.apache.tajo.worker.TajoWorker; @@ -70,7 +72,7 @@ public class QueryMaster extends CompositeService implements EventHandler { private Map queryMasterTasks = Maps.newConcurrentMap(); - private Map finishedQueryMasterTasks = Maps.newConcurrentMap(); + private final LRUMap finishedQueryMasterTasksCache = new LRUMap(HistoryReader.DEFAULT_PAGE_SIZE); private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread; @@ -196,13 +198,14 @@ public QueryMasterTask getQueryMasterTask(QueryId queryId) { return queryMasterTasks.get(queryId); } + @Deprecated public QueryMasterTask getQueryMasterTask(QueryId queryId, boolean includeFinished) { QueryMasterTask queryMasterTask = queryMasterTasks.get(queryId); if(queryMasterTask != null) { return queryMasterTask; } else { if(includeFinished) { - return finishedQueryMasterTasks.get(queryId); + return (QueryMasterTask) finishedQueryMasterTasksCache.get(queryId); } else { return null; } @@ -217,10 +220,9 @@ public Collection getQueryMasterTasks() { return queryMasterTasks.values(); } - //This is not safe OOM @Deprecated public Collection getFinishedQueryMasterTasks() { - return finishedQueryMasterTasks.values(); + return finishedQueryMasterTasksCache.values(); } public class QueryMasterContext { @@ -269,7 +271,7 @@ public void stopQuery(QueryId queryId) { return; } - finishedQueryMasterTasks.put(queryId, queryMasterTask); + finishedQueryMasterTasksCache.put(queryId, queryMasterTask); TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(queryMasterTask); CallFuture future = new CallFuture(); @@ -429,7 +431,7 @@ public void run() { class FinishedQueryMasterTaskCleanThread extends Thread { public void run() { - int expireIntervalTime = systemConf.getIntVar(TajoConf.ConfVars.QUERYMASTER_HISTORY_EXPIRE_PERIOD); + int expireIntervalTime = systemConf.getIntVar(TajoConf.ConfVars.QUERYMASTER_CACHE_EXPIRE_PERIOD); LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime); while(!isStopped) { try { @@ -449,24 +451,25 @@ public void run() { } private void cleanExpiredFinishedQueryMasterTask(long expireTime) { - synchronized(finishedQueryMasterTasks) { + synchronized(finishedQueryMasterTasksCache) { List expiredQueryIds = new ArrayList(); - for(Map.Entry entry: finishedQueryMasterTasks.entrySet()) { - + for(Object key: finishedQueryMasterTasksCache.keySet()) { + QueryId queryId = (QueryId) key; /* If a query are abnormal termination, the finished time will be zero. */ - long finishedTime = entry.getValue().getStartTime(); - Query query = entry.getValue().getQuery(); + QueryMasterTask queryMasterTask = (QueryMasterTask) finishedQueryMasterTasksCache.get(queryId); + long finishedTime = queryMasterTask.getStartTime(); + Query query = queryMasterTask.getQuery(); if (query != null && query.getFinishTime() > 0) { finishedTime = query.getFinishTime(); } if(finishedTime < expireTime) { - expiredQueryIds.add(entry.getKey()); + expiredQueryIds.add(queryId); } } for(QueryId eachId: expiredQueryIds) { - finishedQueryMasterTasks.remove(eachId); + finishedQueryMasterTasksCache.remove(eachId); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index d2a43f32a6..25c7261035 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -206,8 +206,8 @@ public void getExecutionBlockContext(RpcController controller, .setShuffleType(shuffleType); //Set assigned worker to stage - if (!stage.getWorkerMap().containsKey(request.getWorker().getId())) { - stage.getWorkerMap().put(request.getWorker().getId(), + if (!stage.getAssignedWorkerMap().containsKey(request.getWorker().getId())) { + stage.getAssignedWorkerMap().put(request.getWorker().getId(), NetUtils.createSocketAddr(request.getWorker().getHost(), request.getWorker().getPeerRpcPort())); } done.run(ebRequestProto.build()); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index 8247b9fec5..1fcc22eefd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -271,7 +271,7 @@ public void handle(TaskSchedulerEvent event) { protected void killTaskAttempt(int workerId, TaskAttemptId taskAttemptId) { NettyClientBase tajoWorkerRpc; ExecutionBlockId ebId = taskAttemptId.getTaskId().getExecutionBlockId(); - InetSocketAddress workerAddress = getQuery().getStage(ebId).getWorkerMap().get(workerId); + InetSocketAddress workerAddress = getQuery().getStage(ebId).getAssignedWorkerMap().get(workerId); try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(workerAddress, TajoWorkerProtocol.class, true); @@ -474,7 +474,7 @@ public long getQuerySubmitTime() { private void cleanupQuery(final QueryId queryId) { Set workers = Sets.newHashSet(); for (Stage stage : getQuery().getStages()) { - workers.addAll(stage.getWorkerMap().values()); + workers.addAll(stage.getAssignedWorkerMap().values()); } LOG.info("Cleanup resources of all workers. Query: " + queryId + ", workers: " + workers.size()); @@ -485,7 +485,7 @@ public void run() { try { AsyncRpcClient rpc = RpcClientManager.getInstance().getClient(worker, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub(); - tajoWorkerProtocolService.cleanup(null, queryId.getProto(), NullCallback.get()); + tajoWorkerProtocolService.stopQuery(null, queryId.getProto(), NullCallback.get()); } catch (Throwable e) { LOG.error(e.getMessage(), e); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index 5601fdf543..d54766ba00 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -677,13 +677,13 @@ private void stopScheduler() { /** * Get the launched worker address */ - protected Map getWorkerMap() { + protected Map getAssignedWorkerMap() { return workerMap; } private void sendStopExecutionBlockEvent(final TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto) { - for (final InetSocketAddress worker : getWorkerMap().values()) { + for (final InetSocketAddress worker : getAssignedWorkerMap().values()) { getContext().getQueryMasterContext().getEventExecutor().submit(new Runnable() { @Override public void run() { @@ -1071,8 +1071,6 @@ private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOExceptio TableDesc table = stage.context.getTableDescMap().get(scan.getCanonicalName()); Collection fragments; - TableMeta meta = table.getMeta(); - Tablespace tablespace = TablespaceManager.get(scan.getTableDesc().getUri()).get(); // Depending on scanner node's type, it creates fragments. If scan is for diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java index c0a6453a78..2ae4beda16 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java @@ -118,7 +118,7 @@ public GetQueryHistoryResponse getQueryHistory(RpcController controller, QueryId try { QueryId queryId = new QueryId(request.getQueryId()); - QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId, true); + QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId); QueryHistory queryHistory = null; if (queryMasterTask == null) { queryHistory = workerContext.getHistoryReader().getQueryHistory(queryId.toString()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index cf703c370e..0eda37e5fe 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -139,7 +139,7 @@ public void killTaskAttempt(RpcController controller, TajoIdProtos.TaskAttemptId } @Override - public void cleanup(RpcController controller, TajoIdProtos.QueryIdProto request, + public void stopQuery(RpcController controller, TajoIdProtos.QueryIdProto request, RpcCallback done) { workerContext.getTaskManager().getDispatcher().getEventHandler().handle(new QueryStopEvent(new QueryId(request))); diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 439e7f0c85..53c901da07 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -233,7 +233,7 @@ service TajoWorkerProtocolService { rpc allocateTasks(BatchAllocationRequestProto) returns (BatchAllocationResponseProto); rpc stopExecutionBlock(StopExecutionBlockRequestProto) returns (BoolProto); rpc killTaskAttempt(TaskAttemptIdProto) returns (BoolProto); - rpc cleanup(QueryIdProto) returns (BoolProto); + rpc stopQuery(QueryIdProto) returns (BoolProto); } message EnforceProperty { diff --git a/tajo-core/src/main/resources/webapps/admin/query.jsp b/tajo-core/src/main/resources/webapps/admin/query.jsp index ca376bb057..b4210e68a3 100644 --- a/tajo-core/src/main/resources/webapps/admin/query.jsp +++ b/tajo-core/src/main/resources/webapps/admin/query.jsp @@ -57,9 +57,7 @@ } } - String keyword = request.getParameter("keyword"); - HistoryReader historyReader = master.getContext().getHistoryReader(); - List allFinishedQueries = historyReader.getQueries(keyword); + List allFinishedQueries = new ArrayList(master.getContext().getQueryJobManager().getFinishedQueries()); int numOfFinishedQueries = allFinishedQueries.size(); int totalPage = numOfFinishedQueries % pageSize == 0 ? diff --git a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp index eb40b4fdfa..49e5784352 100644 --- a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp +++ b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp @@ -41,7 +41,7 @@ TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext() - .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true); + .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId); boolean runningQuery = queryMasterTask != null; diff --git a/tajo-core/src/main/resources/webapps/worker/queryplan.jsp b/tajo-core/src/main/resources/webapps/worker/queryplan.jsp index 878efe3ef3..422edfc716 100644 --- a/tajo-core/src/main/resources/webapps/worker/queryplan.jsp +++ b/tajo-core/src/main/resources/webapps/worker/queryplan.jsp @@ -37,7 +37,7 @@ TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext() - .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true); + .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId); if(queryMasterTask == null) { out.write(""); diff --git a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp index 0218e13ea9..8385b996cc 100644 --- a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp @@ -24,7 +24,6 @@ <%@ page import="org.apache.tajo.TaskAttemptId" %> <%@ page import="org.apache.tajo.catalog.statistics.TableStats" %> <%@ page import="org.apache.tajo.plan.util.PlannerUtil" %> -<%@ page import="org.apache.tajo.ipc.QueryCoordinatorProtocol" %> <%@ page import="org.apache.tajo.querymaster.*" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> <%@ page import="org.apache.tajo.worker.TajoWorker" %> @@ -34,6 +33,7 @@ <%@ page import="org.apache.tajo.util.*" %> <%@ page import="java.util.*" %> <%@ page import="org.apache.tajo.TajoProtos" %> +<%@ page import="org.apache.tajo.master.cluster.WorkerConnectionInfo" %> <% String paramQueryId = request.getParameter("queryId"); @@ -71,7 +71,7 @@ } } QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext() - .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true); + .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId); if(queryMasterTask == null) { out.write(""); @@ -229,17 +229,12 @@ "&page=" + currentPage + "&pageSize=" + pageSize + "&taskSeq=" + taskSeq + "&sort=" + sort + "&sortOrder=" + sortOrder; - String taskHost = eachTask.getSucceededWorker() == null ? "-" : eachTask.getSucceededWorker().getHost(); - if(eachTask.getSucceededWorker() != null) { - TajoProtos.WorkerConnectionInfoProto worker = - workerMap.get(eachTask.getLastAttempt().getWorkerConnectionInfo().getId()); - if(worker != null) { - TaskAttempt lastAttempt = eachTask.getLastAttempt(); - if(lastAttempt != null) { - TaskAttemptId lastAttemptId = lastAttempt.getId(); - taskHost = "" + eachTask.getSucceededWorker().getHost() + ""; - } - } + TaskAttempt lastAttempt = eachTask.getLastAttempt(); + String taskHost = lastAttempt == null ? "-" : lastAttempt.getWorkerConnectionInfo().getHost(); + if(lastAttempt != null) { + WorkerConnectionInfo conn = lastAttempt.getWorkerConnectionInfo(); + TaskAttemptId lastAttemptId = lastAttempt.getId(); + taskHost = "" + conn.getHost() + ""; } %> diff --git a/tajo-core/src/main/resources/webapps/worker/task.jsp b/tajo-core/src/main/resources/webapps/worker/task.jsp index 32b16498c8..4837589674 100644 --- a/tajo-core/src/main/resources/webapps/worker/task.jsp +++ b/tajo-core/src/main/resources/webapps/worker/task.jsp @@ -56,7 +56,7 @@ int taskSeq = Integer.parseInt(request.getParameter("taskSeq")); TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext() - .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId, true); + .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId); if(queryMasterTask == null) { out.write(""); From 95cf4237edcb52a0f2de3890b6b811929a9578db Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 8 Jul 2015 10:25:45 +0900 Subject: [PATCH 29/80] sort query list --- tajo-core/src/main/resources/webapps/admin/query.jsp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tajo-core/src/main/resources/webapps/admin/query.jsp b/tajo-core/src/main/resources/webapps/admin/query.jsp index b4210e68a3..ae7d910978 100644 --- a/tajo-core/src/main/resources/webapps/admin/query.jsp +++ b/tajo-core/src/main/resources/webapps/admin/query.jsp @@ -58,6 +58,12 @@ } List allFinishedQueries = new ArrayList(master.getContext().getQueryJobManager().getFinishedQueries()); + Collections.sort(allFinishedQueries, new Comparator() { + @Override + public int compare(QueryInfo query1, QueryInfo query2) { + return query2.getQueryId().toString().compareTo(query1.getQueryId().toString()); + } + }); int numOfFinishedQueries = allFinishedQueries.size(); int totalPage = numOfFinishedQueries % pageSize == 0 ? From 2cc47907237500ef45a0fbdf3e06d4cd85d3cfea Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 8 Jul 2015 11:56:40 +0900 Subject: [PATCH 30/80] improve qm distribution --- .../master/scheduler/SimpleScheduler.java | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 7b0321874f..d0e32481ed 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -92,8 +93,10 @@ private void updateResource() { NodeResources.update(maxResource, totalResource); NodeResources.update(clusterResource, resource); - LOG.info("Cluster Resource. available : " + getClusterResource() - + " maximum: " + getMaximumResourceCapability()); + if(LOG.isDebugEnabled()) { + LOG.debug("Cluster Resource. available : " + getClusterResource() + + " maximum: " + getMaximumResourceCapability()); + } } @Override @@ -121,16 +124,29 @@ public ResourceCalculator getResourceCalculator() { private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceRequest(QueryInfo queryInfo) { int qmMemory = tajoConf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); + NodeResource qmResource = NodeResources.createResource(qmMemory); + + //find idle node for QM + Set idleNode = Sets.newHashSet(); + int containers = 1; + for (Worker worker : getRMContext().getWorkers().values()) { + if (worker.getNumRunningQueryMaster() == 0 && NodeResources.fitsIn(qmResource, worker.getAvailableResource())) { + idleNode.add(worker.getWorkerId()); + } + + if (idleNode.size() > containers * 3) break; + } QueryCoordinatorProtocol.NodeResourceRequestProto.Builder builder = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); builder.setQueryId(queryInfo.getQueryId().getProto()) - .setCapacity(NodeResources.createResource(qmMemory).getProto()) + .setCapacity(qmResource.getProto()) .setType(QueryCoordinatorProtocol.ResourceType.QUERYMASTER) .setPriority(1) - .setNumContainers(1) + .setNumContainers(containers) .setRunningTasks(1) + .addAllCandidateNodes(idleNode) .setUserId(queryInfo.getQueryContext().getUser()); //TODO .setQueue(queryInfo.getQueue()); return builder.build(); @@ -330,6 +346,7 @@ public void run() { LOG.info("No Available Resources for QueryMaster :" + queryInfo.getQueryId() + "," + queryInfo); } else { try { + //if QM resource can't be allocated to a node, it should retry boolean started = masterContext.getQueryJobManager().startQueryJob(query.getQueryId(), allocation.get(0)); if(!started) { queryQueue.put(query); From f3feb75b82a5c0f7d81671cfb311b3e2673e1953 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 8 Jul 2015 12:23:52 +0900 Subject: [PATCH 31/80] fix jsp compilation error --- tajo-core/src/main/resources/webapps/admin/query.jsp | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/tajo-core/src/main/resources/webapps/admin/query.jsp b/tajo-core/src/main/resources/webapps/admin/query.jsp index ae7d910978..495e0eef23 100644 --- a/tajo-core/src/main/resources/webapps/admin/query.jsp +++ b/tajo-core/src/main/resources/webapps/admin/query.jsp @@ -58,12 +58,7 @@ } List allFinishedQueries = new ArrayList(master.getContext().getQueryJobManager().getFinishedQueries()); - Collections.sort(allFinishedQueries, new Comparator() { - @Override - public int compare(QueryInfo query1, QueryInfo query2) { - return query2.getQueryId().toString().compareTo(query1.getQueryId().toString()); - } - }); + Collections.sort(allFinishedQueries, java.util.Collections.reverseOrder()); int numOfFinishedQueries = allFinishedQueries.size(); int totalPage = numOfFinishedQueries % pageSize == 0 ? From b8877e1f3339ed56cbc28648d13970557ca53ee8 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 8 Jul 2015 16:14:04 +0900 Subject: [PATCH 32/80] fix redirection bug in tajo web --- .../org/apache/tajo/master/TajoMaster.java | 2 +- .../master/scheduler/SimpleScheduler.java | 5 ++--- .../main/resources/webapps/admin/index.jsp | 4 ++-- .../resources/webapps/admin/querydetail.jsp | 4 +++- .../resources/webapps/admin/querytasks.jsp | 5 ++++- .../src/main/resources/webapps/admin/task.jsp | 5 ++++- .../resources/webapps/worker/querydetail.jsp | 19 +++++-------------- .../resources/webapps/worker/querytasks.jsp | 3 ++- .../main/resources/webapps/worker/task.jsp | 3 ++- 9 files changed, 25 insertions(+), 25 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 6ace05c680..9327c59474 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -203,7 +203,7 @@ public void serviceInit(Configuration conf) throws Exception { super.serviceInit(systemConf); } catch (Throwable t) { t.printStackTrace(); - System.exit(1); + Runtime.getRuntime().halt(-1); } LOG.info("Tajo Master is initialized."); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index d0e32481ed..b249481557 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -20,11 +20,9 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; - import org.apache.tajo.QueryId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.QueryCoordinatorProtocol; @@ -127,7 +125,7 @@ private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceReques NodeResource qmResource = NodeResources.createResource(qmMemory); //find idle node for QM - Set idleNode = Sets.newHashSet(); + List idleNode = Lists.newArrayList(); int containers = 1; for (Worker worker : getRMContext().getWorkers().values()) { if (worker.getNumRunningQueryMaster() == 0 && NodeResources.fitsIn(qmResource, worker.getAvailableResource())) { @@ -137,6 +135,7 @@ private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceReques if (idleNode.size() > containers * 3) break; } + Collections.shuffle(idleNode); QueryCoordinatorProtocol.NodeResourceRequestProto.Builder builder = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp index cf2b20ce7a..e8862212d2 100644 --- a/tajo-core/src/main/resources/webapps/admin/index.jsp +++ b/tajo-core/src/main/resources/webapps/admin/index.jsp @@ -129,8 +129,8 @@ - - + + diff --git a/tajo-core/src/main/resources/webapps/admin/querydetail.jsp b/tajo-core/src/main/resources/webapps/admin/querydetail.jsp index 099301ee3b..99edca4f8e 100644 --- a/tajo-core/src/main/resources/webapps/admin/querydetail.jsp +++ b/tajo-core/src/main/resources/webapps/admin/querydetail.jsp @@ -40,6 +40,8 @@ queryHistory != null ? JSPUtil.sortStageHistories(queryHistory.getStageHistories()) : null; SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + String masterLabel = master.getContext().getTajoMasterService().getBindAddress().getHostName()+ ":" + + master.getContext().getTajoMasterService().getBindAddress().getPort(); %> @@ -52,7 +54,7 @@ <%@ include file="header.jsp"%>
-

Tajo Master: <%=master.getMasterName()%> <%=JSPUtil.getMasterActiveLabel(master.getContext())%>

+

Tajo Master: <%= masterLabel%> <%=JSPUtil.getMasterActiveLabel(master.getContext())%>


<%=queryId%>

<% diff --git a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp index 8449551608..382004d4d6 100644 --- a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp @@ -132,6 +132,9 @@ "&status=" + status + "&sortOrder=" + nextSortOrder + "&sort="; NumberFormat nf = NumberFormat.getInstance(Locale.US); + + String masterLabel = master.getContext().getTajoMasterService().getBindAddress().getHostName()+ ":" + + master.getContext().getTajoMasterService().getBindAddress().getPort(); %> @@ -144,7 +147,7 @@ <%@ include file="header.jsp"%>
-

Tajo Master: <%=master.getMasterName()%> <%=JSPUtil.getMasterActiveLabel(master.getContext())%>

+

Tajo Master: <%=masterLabel%> <%=JSPUtil.getMasterActiveLabel(master.getContext())%>


<%=ebId.toString()%>


diff --git a/tajo-core/src/main/resources/webapps/admin/task.jsp b/tajo-core/src/main/resources/webapps/admin/task.jsp index 36e8075cb4..63e306fafe 100644 --- a/tajo-core/src/main/resources/webapps/admin/task.jsp +++ b/tajo-core/src/main/resources/webapps/admin/task.jsp @@ -106,12 +106,15 @@ shuffleKey = task.getShuffleKey(); shuffleFileName = task.getShuffleFileName(); } + + String masterLabel = master.getContext().getTajoMasterService().getBindAddress().getHostName()+ ":" + + master.getContext().getTajoMasterService().getBindAddress().getPort(); %> <%@ include file="header.jsp"%>
-

Tajo Master: <%=master.getMasterName()%> <%=JSPUtil.getMasterActiveLabel(master.getContext())%>

+

Tajo Master: <%=masterLabel%> <%=JSPUtil.getMasterActiveLabel(master.getContext())%>


<%=ebId%>


diff --git a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp index 49e5784352..29862e68f9 100644 --- a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp +++ b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp @@ -19,14 +19,12 @@ %> <%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%> -<%@ page import="org.apache.commons.lang.math.NumberUtils" %> <%@ page import="org.apache.tajo.QueryId" %> <%@ page import="org.apache.tajo.SessionVars" %> <%@ page import="org.apache.tajo.querymaster.Query" %> <%@ page import="org.apache.tajo.querymaster.QueryMasterTask" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.TajoIdUtils" %> -<%@ page import="org.apache.tajo.util.history.HistoryReader" %> <%@ page import="org.apache.tajo.util.history.QueryHistory" %> <%@ page import="org.apache.tajo.util.history.StageHistory" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> @@ -37,29 +35,21 @@ <% QueryId queryId = TajoIdUtils.parseQueryId(request.getParameter("queryId")); - String startTime = request.getParameter("startTime"); TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object"); QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext() .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId); - boolean runningQuery = queryMasterTask != null; - QueryHistory queryHistory = null; - - Query query = null; + Query query; if (queryMasterTask != null) { query = queryMasterTask.getQuery(); if (query != null) { queryHistory = query.getQueryHistory(); } } else { - HistoryReader reader = tajoWorker.getWorkerContext().getHistoryReader(); - queryHistory = reader.getQueryHistory(queryId.toString(), NumberUtils.toLong(startTime, 0)); - } - - if (!runningQuery && queryHistory == null) { - out.write(""); + String tajoMasterHttp = request.getScheme() + "://" + JSPUtil.getTajoMasterHttpAddr(tajoWorker.getConfig()); + response.sendRedirect(tajoMasterHttp + request.getRequestURI() + "?" + request.getQueryString()); return; } @@ -67,6 +57,7 @@ queryHistory != null ? JSPUtil.sortStageHistories(queryHistory.getStageHistories()) : null; SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + %> @@ -82,7 +73,7 @@

Tajo Worker: <%=tajoWorker.getWorkerContext().getWorkerName()%>


<% -if (runningQuery && query == null) { +if (query == null) { out.write("Query Status: " + queryMasterTask.getState()); String errorMessage = queryMasterTask.getErrorMessage(); if (errorMessage != null && !errorMessage.isEmpty()) { diff --git a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp index 8385b996cc..2c32006d77 100644 --- a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp @@ -74,7 +74,8 @@ .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId); if(queryMasterTask == null) { - out.write(""); + String tajoMasterHttp = request.getScheme() + "://" + JSPUtil.getTajoMasterHttpAddr(tajoWorker.getConfig()); + response.sendRedirect(tajoMasterHttp + request.getRequestURI() + "?" + request.getQueryString()); return; } diff --git a/tajo-core/src/main/resources/webapps/worker/task.jsp b/tajo-core/src/main/resources/webapps/worker/task.jsp index 4837589674..fcab84e62e 100644 --- a/tajo-core/src/main/resources/webapps/worker/task.jsp +++ b/tajo-core/src/main/resources/webapps/worker/task.jsp @@ -59,7 +59,8 @@ .getQueryMasterManagerService().getQueryMaster().getQueryMasterTask(queryId); if(queryMasterTask == null) { - out.write(""); + String tajoMasterHttp = request.getScheme() + "://" + JSPUtil.getTajoMasterHttpAddr(tajoWorker.getConfig()); + response.sendRedirect(tajoMasterHttp + request.getRequestURI() + "?" + request.getQueryString()); return; } From 82fc091e041adb4c82ad9b2f7b34d6004c4a45bf Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 9 Jul 2015 11:07:45 +0900 Subject: [PATCH 33/80] fix qm cleaner bug --- .../querymaster/DefaultTaskScheduler.java | 18 +++++----- .../apache/tajo/querymaster/QueryMaster.java | 33 +++++++++---------- 2 files changed, 24 insertions(+), 27 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 02af7f0a24..3ed5e44dee 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -288,16 +288,14 @@ private void reserveNodeResource() { taskRequests.handle(taskRequestEvent); } - if(responseProto.getResourceCount() == 0) { - if(remainingScheduledObjectNum() == 0) { - // all task is assigned, wait for stopping message - synchronized (schedulingThread){ - schedulingThread.wait(500); - } - } else { - synchronized (schedulingThread){ - schedulingThread.wait(50); - } + if(remainingScheduledObjectNum() == 0) { + // all task is assigned, wait for stopping message + synchronized (schedulingThread){ + schedulingThread.wait(500); + } + } else { + synchronized (schedulingThread){ + schedulingThread.wait(50); } } } catch (Throwable e) { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index b5d49cd169..bc9bbc585e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -72,7 +72,8 @@ public class QueryMaster extends CompositeService implements EventHandler { private Map queryMasterTasks = Maps.newConcurrentMap(); - private final LRUMap finishedQueryMasterTasksCache = new LRUMap(HistoryReader.DEFAULT_PAGE_SIZE); + private final LRUMap + finishedQueryMasterTasksCache = new LRUMap(HistoryReader.DEFAULT_PAGE_SIZE); private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread; @@ -451,27 +452,25 @@ public void run() { } private void cleanExpiredFinishedQueryMasterTask(long expireTime) { - synchronized(finishedQueryMasterTasksCache) { - List expiredQueryIds = new ArrayList(); - for(Object key: finishedQueryMasterTasksCache.keySet()) { - QueryId queryId = (QueryId) key; + List expiredQueryIds = new ArrayList(); + for(Object key: new ArrayList(finishedQueryMasterTasksCache.keySet())) { + QueryId queryId = (QueryId) key; /* If a query are abnormal termination, the finished time will be zero. */ - QueryMasterTask queryMasterTask = (QueryMasterTask) finishedQueryMasterTasksCache.get(queryId); - long finishedTime = queryMasterTask.getStartTime(); - Query query = queryMasterTask.getQuery(); - if (query != null && query.getFinishTime() > 0) { - finishedTime = query.getFinishTime(); - } - - if(finishedTime < expireTime) { - expiredQueryIds.add(queryId); - } + QueryMasterTask queryMasterTask = (QueryMasterTask) finishedQueryMasterTasksCache.get(queryId); + long finishedTime = queryMasterTask.getStartTime(); + Query query = queryMasterTask.getQuery(); + if (query != null && query.getFinishTime() > 0) { + finishedTime = query.getFinishTime(); } - for(QueryId eachId: expiredQueryIds) { - finishedQueryMasterTasksCache.remove(eachId); + if(finishedTime < expireTime) { + expiredQueryIds.add(queryId); } } + + for(QueryId eachId: expiredQueryIds) { + finishedQueryMasterTasksCache.remove(eachId); + } } } } From a8f64264c75ba6a618cc688dbcb3d2183b4b0eed Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 9 Jul 2015 11:45:33 +0900 Subject: [PATCH 34/80] add timeout --- .../apache/tajo/querymaster/DefaultTaskScheduler.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 3ed5e44dee..4a5b2bcde4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -41,10 +41,7 @@ import org.apache.tajo.master.event.TaskSchedulerEvent.EventType; import org.apache.tajo.plan.serder.LogicalNodeSerializer; import org.apache.tajo.resource.NodeResources; -import org.apache.tajo.rpc.AsyncRpcClient; -import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NettyClientBase; -import org.apache.tajo.rpc.RpcClientManager; +import org.apache.tajo.rpc.*; import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.storage.DataLocation; import org.apache.tajo.storage.fragment.FileFragment; @@ -56,6 +53,7 @@ import java.util.*; import java.util.Map.Entry; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; @@ -904,7 +902,8 @@ public void assignToLeafTasks(LinkedList taskRequests) { TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); - TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); + TajoWorkerProtocol.BatchAllocationResponseProto + responseProto = callFuture.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); if (responseProto.getCancellationTaskCount() > 0) { for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); From b91dbe4706029cbb462a71f1d668b5dbb62b7904 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 9 Jul 2015 11:55:15 +0900 Subject: [PATCH 35/80] fix broken unit test --- .../java/org/apache/tajo/worker/TajoWorkerClientService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java index 2ae4beda16..c0a6453a78 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java @@ -118,7 +118,7 @@ public GetQueryHistoryResponse getQueryHistory(RpcController controller, QueryId try { QueryId queryId = new QueryId(request.getQueryId()); - QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId); + QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId, true); QueryHistory queryHistory = null; if (queryMasterTask == null) { queryHistory = workerContext.getHistoryReader().getQueryHistory(queryId.toString()); From 0ef8fbfc42c9492a39773c9b22f8f4eb23ed1ce2 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 9 Jul 2015 15:28:25 +0900 Subject: [PATCH 36/80] cleanup worker when qm failed --- .../master/scheduler/SimpleScheduler.java | 8 +++- .../querymaster/DefaultTaskScheduler.java | 6 ++- .../tajo/worker/ExecutionBlockContext.java | 11 ++++- .../org/apache/tajo/worker/TaskManager.java | 14 ++++++- .../event/ExecutionBlockErrorEvent.java | 41 +++++++++++++++++++ .../tajo/worker/event/TaskManagerEvent.java | 1 + .../apache/tajo/worker/TestTaskExecutor.java | 8 ++-- 7 files changed, 79 insertions(+), 10 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockErrorEvent.java diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index b249481557..efc61680e6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -68,6 +68,7 @@ public SimpleScheduler(TajoMaster.MasterContext context) { super(SimpleScheduler.class.getName()); this.masterContext = context; this.rmContext = context.getResourceManager().getRMContext(); + //Copy default array capacity from PriorityBlockingQueue. this.queryQueue = new PriorityBlockingQueue(11, COMPARATOR); this.queryProcessor = new Thread(new QueryProcessor()); } @@ -127,7 +128,11 @@ private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceReques //find idle node for QM List idleNode = Lists.newArrayList(); int containers = 1; - for (Worker worker : getRMContext().getWorkers().values()) { + ArrayList workers = new ArrayList(); + workers.addAll(getRMContext().getWorkers().values()); + + Collections.shuffle(workers); + for (Worker worker : workers) { if (worker.getNumRunningQueryMaster() == 0 && NodeResources.fitsIn(qmResource, worker.getAvailableResource())) { idleNode.add(worker.getWorkerId()); } @@ -135,7 +140,6 @@ private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceReques if (idleNode.size() > containers * 3) break; } - Collections.shuffle(idleNode); QueryCoordinatorProtocol.NodeResourceRequestProto.Builder builder = QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 4a5b2bcde4..308747aec0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -292,8 +292,10 @@ private void reserveNodeResource() { schedulingThread.wait(500); } } else { - synchronized (schedulingThread){ - schedulingThread.wait(50); + if(responseProto.getResourceCount() == 0) { + synchronized (schedulingThread){ + schedulingThread.wait(50); + } } } } catch (Throwable e) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index cd8777518d..71129fd6ee 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -40,6 +40,7 @@ import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.storage.StorageUtil; import org.apache.tajo.util.Pair; +import org.apache.tajo.worker.event.ExecutionBlockErrorEvent; import java.io.IOException; import java.util.ArrayList; @@ -257,7 +258,15 @@ public void fatalError(TaskAttemptId taskAttemptId, String message) { .setId(taskAttemptId.getProto()) .setErrorMessage(message); - getStub().fatalError(null, builder.build(), NullCallback.get()); + try { + //If QueryMaster does not responding, current execution block should be stop + CallFuture callFuture = new CallFuture(); + getStub().fatalError(callFuture.getController(), builder.build(), callFuture); + callFuture.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); + } catch (Exception e) { + getWorkerContext().getTaskManager().getDispatcher().getEventHandler() + .handle(new ExecutionBlockErrorEvent(taskAttemptId.getTaskId().getExecutionBlockId(), e)); + } } public TajoWorker.WorkerContext getWorkerContext(){ diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index ef6df940a2..d6f00cffac 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -186,8 +186,9 @@ public void handle(TaskManagerEvent event) { } getTaskExecutor().handle(taskStartEvent); } catch (Exception e) { - LOG.error(e.getMessage(), e); getTaskExecutor().releaseResource(taskStartEvent.getTaskAttemptId()); + getWorkerContext().getTaskManager().getDispatcher().getEventHandler() + .handle(new ExecutionBlockErrorEvent(taskStartEvent.getExecutionBlockId(), e)); } break; } @@ -216,6 +217,17 @@ public void handle(TaskManagerEvent event) { workerContext.cleanup(queryStopEvent.getQueryId().toString()); break; } + case EB_FAIL: { + ExecutionBlockErrorEvent errorEvent = TUtil.checkTypeAndGet(event, ExecutionBlockErrorEvent.class); + LOG.error(errorEvent.getError().getMessage(), errorEvent.getError()); + ExecutionBlockContext context = executionBlockContextMap.remove(errorEvent.getExecutionBlockId()); + + if(context != null) { + context.getSharedResource().releaseBroadcastCache(context.getExecutionBlockId()); + getWorkerContext().getTaskHistoryWriter().flushTaskHistories(); + context.stop(); + } + } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockErrorEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockErrorEvent.java new file mode 100644 index 0000000000..dfc54abf1c --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockErrorEvent.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.worker.event; + +import org.apache.tajo.ExecutionBlockId; + +public class ExecutionBlockErrorEvent extends TaskManagerEvent { + + private ExecutionBlockId executionBlockId; + private Throwable error; + + public ExecutionBlockErrorEvent(ExecutionBlockId executionBlockId, Throwable e) { + super(EventType.EB_FAIL); + this.executionBlockId = executionBlockId; + this.error = e; + } + + public ExecutionBlockId getExecutionBlockId() { + return executionBlockId; + } + + public Throwable getError() { + return error; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java index e1a945cc81..7225e70a3a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java @@ -29,6 +29,7 @@ public enum EventType { //cleanup events EB_STOP, + EB_FAIL, QUERY_STOP } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index ae60d2076d..ca19f6f5f9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -50,7 +50,7 @@ public class TestTaskExecutor { private NodeResourceManager resourceManager; private NodeStatusUpdater statusUpdater; private TaskManager taskManager; - private TaskExecutor taskExecutor; + private MyTaskExecutor taskExecutor; private AsyncDispatcher dispatcher; private AsyncDispatcher taskDispatcher; private TajoWorker.WorkerContext workerContext; @@ -102,7 +102,7 @@ public WorkerConnectionInfo getConnectionInfo() { barrier = new Semaphore(0); resourceManagerBarrier = new Semaphore(0); taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext); - taskExecutor = new TaskExecutor(barrier, workerContext); + taskExecutor = new MyTaskExecutor(barrier, workerContext); resourceManager = new MockNodeResourceManager(resourceManagerBarrier, dispatcher, workerContext); statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext); @@ -192,11 +192,11 @@ public void testTaskException() throws Exception { assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); } - class TaskExecutor extends MockTaskExecutor { + class MyTaskExecutor extends MockTaskExecutor { int completeTasks; AtomicBoolean throwException = new AtomicBoolean(); - public TaskExecutor(Semaphore barrier, TajoWorker.WorkerContext workerContext) { + public MyTaskExecutor(Semaphore barrier, TajoWorker.WorkerContext workerContext) { super(barrier, workerContext); } From 354df26e42fd9c92ab2fb36e4410d9680e078272 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 9 Jul 2015 16:38:45 +0900 Subject: [PATCH 37/80] add missing flag --- .../org/apache/tajo/master/scheduler/SimpleScheduler.java | 6 ++++++ .../main/java/org/apache/tajo/worker/NodeStatusUpdater.java | 5 ++--- .../src/test/java/org/apache/tajo/TajoTestingCluster.java | 3 ++- 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index efc61680e6..7728548195 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -111,6 +111,12 @@ protected void serviceStart() throws Exception { super.serviceStart(); } + @Override + protected void serviceStop() throws Exception { + this.isStopped = true; + super.serviceStop(); + } + @Override public int getRunningQuery() { return assignedQueryMasterMap.size(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index bbf323cd83..8893384ec9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -81,7 +81,7 @@ public void serviceInit(Configuration conf) throws Exception { this.workerContext.getNodeResourceManager().getDispatcher().register(NodeStatusEvent.EventType.class, this); this.heartBeatInterval = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL); this.updaterThread = new StatusUpdaterThread(); - + this.updaterThread.setName("NodeStatusUpdater"); super.serviceInit(conf); } @@ -102,10 +102,9 @@ public void serviceStart() throws Exception { @Override public void serviceStop() throws Exception { this.isStopped = true; - synchronized (updaterThread) { updaterThread.interrupt(); - updaterThread.join(); + updaterThread.join(100); } super.serviceStop(); LOG.info("NodeStatusUpdater stopped."); diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 56afe149c1..57f22991f7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -501,7 +501,8 @@ public void startMiniCluster(final int numSlaves, final String [] dataNodeHosts) startMiniDFSCluster(numDataNodes, clusterTestBuildDir, dataNodeHosts); this.dfsCluster.waitClusterUp(); - conf.setInt("hbase.hconnection.threads.core", 50); + conf.setInt("hbase.hconnection.threads.core", 5); + conf.setInt("hbase.hconnection.threads.max", 50); hbaseUtil = new HBaseTestClusterUtil(conf, clusterTestBuildDir); startMiniTajoCluster(this.clusterTestBuildDir, numSlaves, false); From eba833e3a977511cce9114e32b759eec875f59fb Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 9 Jul 2015 19:05:36 +0900 Subject: [PATCH 38/80] TAJO-1397 --- .../master/scheduler/SimpleScheduler.java | 40 +- .../org/apache/tajo/TajoTestingCluster.java | 3 - .../master/rm/TestTajoResourceManager.java | 416 ------------------ .../master/scheduler/TestSimpleScheduler.java | 74 ++++ 4 files changed, 101 insertions(+), 432 deletions(-) delete mode 100644 tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 7728548195..10983eac87 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -18,8 +18,10 @@ package org.apache.tajo.master.scheduler; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -64,15 +66,20 @@ public class SimpleScheduler extends AbstractQueryScheduler { private final Thread queryProcessor; private TajoConf tajoConf; - public SimpleScheduler(TajoMaster.MasterContext context) { + @VisibleForTesting + public SimpleScheduler(TajoMaster.MasterContext context, TajoRMContext rmContext) { super(SimpleScheduler.class.getName()); this.masterContext = context; - this.rmContext = context.getResourceManager().getRMContext(); + this.rmContext = rmContext; //Copy default array capacity from PriorityBlockingQueue. this.queryQueue = new PriorityBlockingQueue(11, COMPARATOR); this.queryProcessor = new Thread(new QueryProcessor()); } + public SimpleScheduler(TajoMaster.MasterContext context) { + this(context, context.getResourceManager().getRMContext()); + } + private void initScheduler(TajoConf conf) { int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); this.minResource.setMemory(minQMMem).setVirtualCores(1); @@ -131,15 +138,15 @@ private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceReques int qmMemory = tajoConf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); NodeResource qmResource = NodeResources.createResource(qmMemory); - //find idle node for QM - List idleNode = Lists.newArrayList(); + int containers = 1; - ArrayList workers = new ArrayList(); - workers.addAll(getRMContext().getWorkers().values()); + Set assignedQMNodes = Sets.newHashSet(assignedQueryMasterMap.values()); + List idleNode = Lists.newArrayList(); - Collections.shuffle(workers); - for (Worker worker : workers) { - if (worker.getNumRunningQueryMaster() == 0 && NodeResources.fitsIn(qmResource, worker.getAvailableResource())) { + for (Worker worker : getRMContext().getWorkers().values()) { + + //find idle node for QM + if (!assignedQMNodes.contains(worker.getWorkerId())) { idleNode.add(worker.getWorkerId()); } @@ -181,6 +188,7 @@ public int getNumClusterNodes() { if (request.getCandidateNodesCount() > 0) { workers.addAll(request.getCandidateNodesList()); + Collections.shuffle(workers); } int requiredContainers = request.getNumContainers(); @@ -190,7 +198,6 @@ public int getNumClusterNodes() { // reserve resource in random workers if (reservedResources.size() < requiredContainers) { LinkedList randomWorkers = new LinkedList(getRMContext().getWorkers().keySet()); - randomWorkers.removeAll(workers); Collections.shuffle(randomWorkers); reservedResources.addAll(reserveClusterResource( @@ -281,6 +288,10 @@ public void submitQuery(QuerySchedulingInfo schedulingInfo) { pendingQueryMap.put(schedulingInfo.getQueryId(), schedulingInfo); } + protected boolean startQuery(QueryId queryId, AllocationResourceProto allocation) { + return masterContext.getQueryJobManager().startQueryJob(queryId, allocation); + } + public void stopQuery(QueryId queryId) { if(pendingQueryMap.containsKey(queryId)){ queryQueue.remove(pendingQueryMap.remove(queryId)); @@ -307,6 +318,10 @@ public WorkerConnectionInfo getQueryMaster(QueryId queryId) { return null; } + protected QueryInfo getQueryInfo(QueryId queryId) { + return masterContext.getQueryJobManager().getQueryInProgress(queryId).getQueryInfo(); + } + private final class QueryProcessor implements Runnable { @Override public void run() { @@ -338,8 +353,7 @@ public void run() { } } } else { - QueryInfo queryInfo = - masterContext.getQueryJobManager().getQueryInProgress(query.getQueryId()).getQueryInfo(); + QueryInfo queryInfo = getQueryInfo(query.getQueryId()); List allocation = reserve(query.getQueryId(), createQMResourceRequest(queryInfo)); @@ -356,7 +370,7 @@ public void run() { } else { try { //if QM resource can't be allocated to a node, it should retry - boolean started = masterContext.getQueryJobManager().startQueryJob(query.getQueryId(), allocation.get(0)); + boolean started = startQuery(query.getQueryId(), allocation.get(0)); if(!started) { queryQueue.put(query); } else { diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 57f22991f7..71430a3e6d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -147,9 +147,6 @@ void initPropertiesAndConfigs() { conf.setIntVar(ConfVars.CATALOG_RPC_SERVER_WORKER_THREAD_NUM, 2); conf.setIntVar(ConfVars.SHUFFLE_RPC_SERVER_WORKER_THREAD_NUM, 2); - // Resource allocator - conf.setIntVar(ConfVars.$QUERY_EXECUTE_PARALLEL_MAX, 10); - // Memory cache termination conf.setIntVar(ConfVars.WORKER_HISTORY_EXPIRE_PERIOD, 1); diff --git a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java deleted file mode 100644 index de02c281e4..0000000000 --- a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java +++ /dev/null @@ -1,416 +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.protobuf.RpcCallback; -import org.apache.tajo.QueryId; -import org.apache.tajo.QueryIdFactory; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import static org.junit.Assert.*; - -public class TestTajoResourceManager { - private final PrimitiveProtos.BoolProto BOOL_TRUE = PrimitiveProtos.BoolProto.newBuilder().setValue(true).build(); - private final PrimitiveProtos.BoolProto BOOL_FALSE = PrimitiveProtos.BoolProto.newBuilder().setValue(false).build(); - - TajoConf tajoConf; - - long queryIdTime = System.currentTimeMillis(); - int numWorkers = 5; - float workerDiskSlots = 5.0f; - int workerMemoryMB = 512 * 10; - - private TajoResourceManager initResourceManager() throws Exception { - tajoConf = new org.apache.tajo.conf.TajoConf(); - - // tajoConf.setFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT, 0.0f); - tajoConf.setIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY, 512); - tajoConf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS, "localhost:0"); - TajoResourceManager tajoResourceManager = new TajoResourceManager(tajoConf); - tajoResourceManager.init(tajoConf); - tajoResourceManager.start(); - - for(int i = 0; i < numWorkers; i++) { - -// WorkerConnectionInfo connectionInfo = -// new WorkerConnectionInfo("host" + (i + 1), 28091, 28092, 21000 + i, 28093, 28080); -// NodeHeartbeat tajoHeartbeat = NodeHeartbeat.newBuilder() -// .setConnectionInfo(connectionInfo.getProto()) -// .setServerStatus(serverStatus) -// .build(); -// -// tajoResourceManager.getResourceTracker().heartbeat(null, tajoHeartbeat, NullCallback.get()); - } - - return tajoResourceManager; - } - - /* - @Test - public void testHeartbeat() throws Exception { - TajoResourceManager tajoResourceManager = null; - try { - tajoResourceManager = initResourceManager(); - assertEquals(numWorkers, tajoResourceManager.getWorkers().size()); - for(Worker worker: tajoResourceManager.getWorkers().values()) { -// WorkerResource resource = worker.getResource(); -// assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); -// assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); - } - } finally { - if (tajoResourceManager != null) { - tajoResourceManager.stop(); - } - } - } - - @Test - public void testMemoryResource() throws Exception { - TajoResourceManager tajoResourceManager = null; - try { - tajoResourceManager = initResourceManager(); - - final int minMemory = 256; - final int maxMemory = 512; - float diskSlots = 1.0f; - - QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 1); - - WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder() - .setResourceRequestPriority(ResourceRequestPriority.MEMORY) - .setNumContainers(60) - .setQueryId(queryId.getProto()) - .setMaxDiskSlotPerContainer(diskSlots) - .setMinDiskSlotPerContainer(diskSlots) - .setMinMemoryMBPerContainer(minMemory) - .setMaxMemoryMBPerContainer(maxMemory) - .build(); - - final CountDownLatch barrier = new CountDownLatch(1); - final List containerIds = new - ArrayList(); - - RpcCallback callBack = new RpcCallback() { - - @Override - public void run(WorkerResourceAllocationResponse response) { - TestTajoResourceManager.this.response = response; - barrier.countDown(); - } - }; - - tajoResourceManager.allocateWorkerResources(request, callBack); - assertTrue(barrier.await(3, TimeUnit.SECONDS)); - - - // assert after callback - int totalUsedMemory = 0; - int totalUsedDisks = 0; - - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - assertEquals(0, resource.getAvailableMemoryMB()); - assertEquals(0, resource.getAvailableDiskSlots(), 0); - assertEquals(5.0f, resource.getUsedDiskSlots(), 0); - - totalUsedMemory += resource.getUsedMemoryMB(); - totalUsedDisks += resource.getUsedDiskSlots(); - } - - assertEquals(workerMemoryMB * numWorkers, totalUsedMemory); - assertEquals(workerDiskSlots * numWorkers, totalUsedDisks, 0); - - assertEquals(numWorkers * 10, response.getWorkerAllocatedResourceList().size()); - - for(WorkerAllocatedResource eachResource: response.getWorkerAllocatedResourceList()) { - assertTrue( - eachResource.getAllocatedMemoryMB() >= minMemory && eachResource.getAllocatedMemoryMB() <= maxMemory); - containerIds.add(eachResource.getContainerId()); - } - - for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) { - tajoResourceManager.releaseWorkerResource(eachContainerId); - } - - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); - assertEquals(0, resource.getUsedMemoryMB()); - - assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); - assertEquals(0.0f, resource.getUsedDiskSlots(), 0); - } - } finally { - if (tajoResourceManager != null) { - tajoResourceManager.stop(); - } - } - } - - @Test - public void testMemoryNotCommensurable() throws Exception { - TajoResourceManager tajoResourceManager = null; - - try { - tajoResourceManager = initResourceManager(); - - final int minMemory = 200; - final int maxMemory = 500; - float diskSlots = 1.0f; - - QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 2); - - int requiredContainers = 60; - - int numAllocatedContainers = 0; - - int loopCount = 0; - while(true) { - WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder() - .setResourceRequestPriority(ResourceRequestPriority.MEMORY) - .setNumContainers(requiredContainers - numAllocatedContainers) - .setQueryId(queryId.getProto()) - .setMaxDiskSlotPerContainer(diskSlots) - .setMinDiskSlotPerContainer(diskSlots) - .setMinMemoryMBPerContainer(minMemory) - .setMaxMemoryMBPerContainer(maxMemory) - .build(); - - final CountDownLatch barrier = new CountDownLatch(1); - - RpcCallback callBack = new RpcCallback() { - @Override - public void run(WorkerResourceAllocationResponse response) { - TestTajoResourceManager.this.response = response; - barrier.countDown(); - } - }; - - tajoResourceManager.allocateWorkerResources(request, callBack); - - assertTrue(barrier.await(3, TimeUnit.SECONDS)); - - numAllocatedContainers += TestTajoResourceManager.this.response.getWorkerAllocatedResourceList().size(); - - //release resource - for(WorkerAllocatedResource eachResource: - TestTajoResourceManager.this.response.getWorkerAllocatedResourceList()) { - assertTrue( - eachResource.getAllocatedMemoryMB() >= minMemory && eachResource.getAllocatedMemoryMB() <= maxMemory); - tajoResourceManager.releaseWorkerResource(eachResource.getContainerId()); - } - - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - assertEquals(0, resource.getUsedMemoryMB()); - assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); - - assertEquals(0.0f, resource.getUsedDiskSlots(), 0); - assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); - } - - loopCount++; - - if(loopCount == 2) { - assertEquals(requiredContainers, numAllocatedContainers); - break; - } - } - - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - assertEquals(0, resource.getUsedMemoryMB()); - assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); - - assertEquals(0.0f, resource.getUsedDiskSlots(), 0); - assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); - } - } finally { - if (tajoResourceManager != null) { - tajoResourceManager.stop(); - } - } - } - - @Test - public void testDiskResource() throws Exception { - TajoResourceManager tajoResourceManager = null; - - try { - tajoResourceManager = initResourceManager(); - - final float minDiskSlots = 1.0f; - final float maxDiskSlots = 2.0f; - int memoryMB = 256; - - QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 3); - - WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder() - .setResourceRequestPriority(ResourceRequestPriority.DISK) - .setNumContainers(60) - .setQueryId(queryId.getProto()) - .setMaxDiskSlotPerContainer(maxDiskSlots) - .setMinDiskSlotPerContainer(minDiskSlots) - .setMinMemoryMBPerContainer(memoryMB) - .setMaxMemoryMBPerContainer(memoryMB) - .build(); - - final CountDownLatch barrier = new CountDownLatch(1); - final List containerIds = new - ArrayList(); - - - RpcCallback callBack = new RpcCallback() { - - @Override - public void run(WorkerResourceAllocationResponse response) { - TestTajoResourceManager.this.response = response; - barrier.countDown(); - } - }; - - tajoResourceManager.allocateWorkerResources(request, callBack); - assertTrue(barrier.await(3, TimeUnit.SECONDS)); - - for(WorkerAllocatedResource eachResource: response.getWorkerAllocatedResourceList()) { - assertTrue("AllocatedDiskSlot:" + eachResource.getAllocatedDiskSlots(), - eachResource.getAllocatedDiskSlots() >= minDiskSlots && - eachResource.getAllocatedDiskSlots() <= maxDiskSlots); - containerIds.add(eachResource.getContainerId()); - } - - // assert after callback - int totalUsedDisks = 0; - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - //each worker allocated 3 container (2 disk slot = 2, 1 disk slot = 1) - assertEquals(0, resource.getAvailableDiskSlots(), 0); - assertEquals(5.0f, resource.getUsedDiskSlots(), 0); - assertEquals(256 * 3, resource.getUsedMemoryMB()); - - totalUsedDisks += resource.getUsedDiskSlots(); - } - - assertEquals(workerDiskSlots * numWorkers, totalUsedDisks, 0); - - assertEquals(numWorkers * 3, response.getWorkerAllocatedResourceList().size()); - - for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) { - tajoResourceManager.releaseWorkerResource(eachContainerId); - } - - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); - assertEquals(0, resource.getUsedMemoryMB()); - - assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); - assertEquals(0.0f, resource.getUsedDiskSlots(), 0); - } - } finally { - if (tajoResourceManager != null) { - tajoResourceManager.stop(); - } - } - } - - @Test - public void testDiskResourceWithStoppedQuery() throws Exception { - TajoResourceManager tajoResourceManager = null; - - try { - tajoResourceManager = initResourceManager(); - - final float minDiskSlots = 1.0f; - final float maxDiskSlots = 2.0f; - int memoryMB = 256; - - QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 3); - - WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder() - .setResourceRequestPriority(ResourceRequestPriority.DISK) - .setNumContainers(60) - .setQueryId(queryId.getProto()) - .setMaxDiskSlotPerContainer(maxDiskSlots) - .setMinDiskSlotPerContainer(minDiskSlots) - .setMinMemoryMBPerContainer(memoryMB) - .setMaxMemoryMBPerContainer(memoryMB) - .build(); - - final CountDownLatch barrier = new CountDownLatch(1); - final List containerIds = new - ArrayList(); - - - RpcCallback callBack = new RpcCallback() { - - @Override - public void run(WorkerResourceAllocationResponse response) { - TestTajoResourceManager.this.response = response; - barrier.countDown(); - } - }; - - tajoResourceManager.getRMContext().getStoppedQueryIds().add(queryId); - tajoResourceManager.allocateWorkerResources(request, callBack); - assertFalse(barrier.await(3, TimeUnit.SECONDS)); - - assertNull(response); - - // assert after callback - int totalUsedDisks = 0; - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - //each worker allocated 3 container (2 disk slot = 2, 1 disk slot = 1) - assertEquals(5.0f, resource.getAvailableDiskSlots(), 0); - assertEquals(0, resource.getUsedDiskSlots(), 0); - assertEquals(0, resource.getUsedMemoryMB()); - - totalUsedDisks += resource.getUsedDiskSlots(); - } - - assertEquals(0, totalUsedDisks, 0); - - for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) { - tajoResourceManager.releaseWorkerResource(eachContainerId); - } - - for(Worker worker: tajoResourceManager.getWorkers().values()) { - WorkerResource resource = worker.getResource(); - assertEquals(workerMemoryMB, resource.getAvailableMemoryMB()); - assertEquals(0, resource.getUsedMemoryMB()); - - assertEquals(workerDiskSlots, resource.getAvailableDiskSlots(), 0); - assertEquals(0.0f, resource.getUsedDiskSlots(), 0); - } - } finally { - if (tajoResourceManager != null) { - tajoResourceManager.stop(); - } - } - } - */ -} diff --git a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java new file mode 100644 index 0000000000..c21a19939c --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java @@ -0,0 +1,74 @@ +/** + * 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.scheduler; + +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.tajo.QueryId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.master.QueryInfo; +import org.apache.tajo.master.rm.TajoRMContext; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestSimpleScheduler { + SimpleScheduler scheduler; + TajoRMContext rmContext; + AsyncDispatcher dispatcher; + TajoConf conf; + + @Before + public void setup() { + conf = new TajoConf(); + dispatcher = new AsyncDispatcher(); + dispatcher.init(conf); + dispatcher.start(); + + rmContext = new TajoRMContext(dispatcher); + scheduler = new MySimpleScheduler(rmContext); + } + + @After + public void tearDown() { + scheduler.stop(); + dispatcher.stop(); + } + + @Test + public void testReserveResource() { + } + + class MySimpleScheduler extends SimpleScheduler { + + public MySimpleScheduler(TajoRMContext rmContext) { + super(null, rmContext); + } + + @Override + protected boolean startQuery(QueryId queryId, QueryCoordinatorProtocol.AllocationResourceProto allocation) { + return true; + } + + @Override + protected QueryInfo getQueryInfo(QueryId queryId) { + return new QueryInfo(queryId); + } + } +} From 347d3bfe90adbccb26d898f861e70a3da155b0e5 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 10 Jul 2015 19:48:49 +0900 Subject: [PATCH 39/80] fix race condition --- .../java/org/apache/tajo/util/StringUtils.java | 9 ++++++--- .../apache/tajo/engine/parser/SQLAnalyzer.java | 16 ++++++---------- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java index 0a160725f7..b8f3254ee9 100644 --- a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java +++ b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java @@ -18,6 +18,7 @@ package org.apache.tajo.util; +import io.netty.util.CharsetUtil; import org.apache.commons.lang.CharUtils; import org.apache.commons.lang.StringEscapeUtils; import org.apache.commons.lang.SystemUtils; @@ -77,10 +78,12 @@ public static String formatTime(long timeDiff){ } return buf.toString(); } - - static CharsetEncoder asciiEncoder = Charset.forName("US-ASCII").newEncoder(); // or "ISO-8859-1" for ISO Latin 1 - + /** + * or "ISO-8859-1" for ISO Latin 1 + */ public static boolean isPureAscii(String v) { + // get thread-safe encoder + CharsetEncoder asciiEncoder = CharsetUtil.getEncoder(CharsetUtil.US_ASCII); return asciiEncoder.canEncode(v); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java index 62bb0f9533..582fa140b8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java @@ -28,9 +28,7 @@ import org.apache.tajo.algebra.*; import org.apache.tajo.algebra.Aggregation.GroupType; import org.apache.tajo.algebra.LiteralValue.LiteralType; -import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.engine.parser.SQLParser.*; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.util.StringUtils; @@ -44,7 +42,6 @@ import static org.apache.tajo.engine.parser.SQLParser.*; public class SQLAnalyzer extends SQLParserBaseVisitor { - private SQLParser parser; public SQLAnalyzer() { } @@ -53,15 +50,14 @@ public Expr parse(String sql) { ANTLRInputStream input = new ANTLRInputStream(sql); SQLLexer lexer = new SQLLexer(input); CommonTokenStream tokens = new CommonTokenStream(lexer); - this.parser = new SQLParser(tokens); - parser.setBuildParseTree(true); - parser.removeErrorListeners(); - - parser.setErrorHandler(new SQLErrorStrategy()); - parser.addErrorListener(new SQLErrorListener()); - SqlContext context; try { + SQLParser parser = new SQLParser(tokens); + parser.setBuildParseTree(true); + parser.removeErrorListeners(); + + parser.setErrorHandler(new SQLErrorStrategy()); + parser.addErrorListener(new SQLErrorListener()); context = parser.sql(); } catch (SQLParseError e) { e.printStackTrace(); From 23ec8b665fef6b2c798eb774e096eff62b97f708 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 10 Jul 2015 19:49:50 +0900 Subject: [PATCH 40/80] add failure handling --- .../src/main/java/org/apache/tajo/querymaster/Query.java | 6 +++++- .../main/java/org/apache/tajo/querymaster/TaskAttempt.java | 4 ++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java index 16c4a1f255..ec57aa8b89 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java @@ -177,7 +177,11 @@ public class Query implements EventHandler { // Transitions from KILLED state // ignore-able transitions .addTransition(QueryState.QUERY_KILLED, QueryState.QUERY_KILLED, - EnumSet.of(QueryEventType.QUERY_COMPLETED, QueryEventType.KILL)) + EnumSet.of(QueryEventType.START, QueryEventType.QUERY_COMPLETED, + QueryEventType.KILL, QueryEventType.INTERNAL_ERROR)) + .addTransition(QueryState.QUERY_KILLED, QueryState.QUERY_ERROR, + QueryEventType.INTERNAL_ERROR, + INTERNAL_ERROR_TRANSITION) // Transitions from FAILED state .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED, diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java index b650089632..5e68711506 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java @@ -167,6 +167,10 @@ TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition()) TaskAttemptEventType.TA_ASSIGNED, TaskAttemptEventType.TA_DONE), new TaskKilledCompleteTransition()) + + // Transitions from TA_FAILED state + .addTransition(TaskAttemptState.TA_FAILED, TaskAttemptState.TA_FAILED, + TaskAttemptEventType.TA_KILL) .installTopology(); private final StateMachine From cdeee9b4fc7cbb0ce0365ce178a5b23a71f45cea Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 10 Jul 2015 19:58:05 +0900 Subject: [PATCH 41/80] refactor same table name for parallel testing --- .../engine/codegen/TestEvalCodeGenerator.java | 326 ++--- .../tajo/engine/eval/TestPredicates.java | 134 +- .../tajo/engine/eval/TestSQLExpression.java | 1198 ++++++++--------- .../engine/function/TestBuiltinFunctions.java | 148 +- .../function/TestConditionalExpressions.java | 84 +- .../function/TestDateTimeFunctions.java | 57 +- .../engine/function/TestMathFunctions.java | 56 +- .../TestPatternMatchingPredicates.java | 30 +- .../TestStringOperatorsAndFunctions.java | 110 +- .../planner/physical/TestBNLJoinExec.java | 66 +- .../physical/TestFullOuterHashJoinExec.java | 88 +- .../physical/TestFullOuterMergeJoinExec.java | 94 +- .../physical/TestLeftOuterHashJoinExec.java | 87 +- .../physical/TestRightOuterHashJoinExec.java | 73 +- .../physical/TestRightOuterMergeJoinExec.java | 68 +- .../tajo/engine/query/TestCreateTable.java | 82 +- .../tajo/engine/query/TestHBaseTable.java | 429 +++--- .../apache/tajo/ha/TestHAServiceHDFSImpl.java | 1 + .../tajo/querymaster/TestKillQuery.java | 2 + .../org/apache/tajo/worker/TestFetcher.java | 10 +- .../quoted_identifier_mixed_chars_1.sql | 2 +- .../quoted_identifier_mixed_chars_2.sql | 2 +- .../quoted_identifier_mixed_chars_3.sql | 4 +- .../TestCreateTable/testNestedRecord2.sql | 2 +- 24 files changed, 1599 insertions(+), 1554 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java b/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java index d86081af8a..279c118003 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java @@ -45,22 +45,22 @@ public class TestEvalCodeGenerator extends ExprTestBase { @Test public void testArithmetic() throws IOException { - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1+1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col2 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col3 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col4 from table1;", new String [] {"5.5"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col5 from table1;", new String [] {"6.5"}); + testEval(schema, "testarithmetic", "0,1,2,3,4.5,6.5", "select 1+1;", new String [] {"2"}); + testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col2 from testarithmetic;", new String [] {"3"}); + testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col3 from testarithmetic;", new String [] {"4"}); + testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col4 from testarithmetic;", new String [] {"5.5"}); + testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col5 from testarithmetic;", new String [] {"6.5"}); } @Test public void testGetField() throws IOException { - testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col1 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col2 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col3 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col4 from table1;", new String [] {"4.5"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col5 from table1;", new String [] {"5.5"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col6 from table1;", new String [] {"F6"}); - testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6,abc,t", "select col8 from table1;", new String [] {"t"}); + testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col1 from testgetfield;", new String [] {"1"}); + testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col2 from testgetfield;", new String [] {"2"}); + testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col3 from testgetfield;", new String [] {"3"}); + testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col4 from testgetfield;", new String [] {"4.5"}); + testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col5 from testgetfield;", new String [] {"5.5"}); + testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col6 from testgetfield;", new String [] {"F6"}); + testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6,abc,t", "select col8 from testgetfield;", new String [] {"t"}); } @Test @@ -78,29 +78,29 @@ public void testNullHandling() throws IOException { schema.addColumn("col9", TajoDataTypes.Type.BOOLEAN); schema.addColumn("nullable", TajoDataTypes.Type.NULL_TYPE); - testEval(schema, "table1", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is null from table1;", new String[]{"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is null from table1;", new String [] {"t"}); - - testEval(schema, "table1", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is not null from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is not null from table1;", new String [] {"f"}); + testEval(schema, "testnullhandling", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is null from testnullhandling;", new String[]{"t"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is null from testnullhandling;", new String [] {"t"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is null from testnullhandling;", new String [] {"t"}); + + testEval(schema, "testnullhandling", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is not null from testnullhandling;", new String [] {"f"}); } @Test @@ -119,44 +119,44 @@ public void testComparison() throws IOException { testSimpleEval("select 1 < null;", new String [] {""}); testSimpleEval("select 1 <= null;", new String [] {""}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col5 from table1;", new String [] {"f"}); - - testEval(inetSchema, "table1", "192.168.0.1,192.168.0.1", "select addr1 = addr2 from table1;", new String[]{"t"}); - testEval(inetSchema, "table1", "192.168.0.1,192.168.0.2", "select addr1 = addr2 from table1;", new String[]{"f"}); - - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col5 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col5 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col5 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col5 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col5 from table1;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col1 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col2 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col3 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col4 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col5 from testcomparison;", new String [] {"f"}); + + testEval(inetSchema, "testcomparison", "192.168.0.1,192.168.0.1", "select addr1 = addr2 from testcomparison;", new String[]{"t"}); + testEval(inetSchema, "testcomparison", "192.168.0.1,192.168.0.2", "select addr1 = addr2 from testcomparison;", new String[]{"f"}); + + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col1 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col2 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col3 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col4 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col5 from testcomparison;", new String [] {"t"}); + + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col1 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col2 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col3 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col4 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col5 from testcomparison;", new String [] {"t"}); + + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col1 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col2 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col3 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col4 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col5 from testcomparison;", new String [] {"t"}); + + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col1 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col2 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col3 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col4 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col5 from testcomparison;", new String [] {"f"}); + + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col1 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col2 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col3 from testcomparison;", new String [] {"t"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col4 from testcomparison;", new String [] {"f"}); + testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col5 from testcomparison;", new String [] {"f"}); } @Test @@ -164,33 +164,33 @@ public void testBetweenAsymmetric() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TajoDataTypes.Type.INT4); schema.addColumn("col2", TajoDataTypes.Type.INT4); - testEval(schema, "table1", "0,", "select col1 between 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "1,", "select col1 between 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "2,", "select col1 between 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "3,", "select col1 between 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "4,", "select col1 between 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "5,", "select (col2 between 1 and 3) is null from table1", new String[]{"t"}); - - testEval(schema, "table1", "0,", "select col1 between 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "1,", "select col1 between 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "2,", "select col1 between 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "3,", "select col1 between 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "4,", "select col1 between 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "5,", "select (col2 between 3 and 1) is null from table1", new String[]{"t"}); - - testEval(schema, "table1", "0,", "select col1 not between 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "1,", "select col1 not between 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "2,", "select col1 not between 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "3,", "select col1 not between 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "4,", "select col1 not between 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "5,", "select (col2 not between 1 and 3) is null from table1", new String[]{"t"}); - - testEval(schema, "table1", "0,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "1,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "2,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "3,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "4,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "5,", "select (col2 not between 3 and 1) is null from table1", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "0,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "1,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "2,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "3,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "4,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "5,", "select (col2 between 1 and 3) is null from testbetweenasymmetric", new String[]{"t"}); + + testEval(schema, "testbetweenasymmetric", "0,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "1,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "2,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "3,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "4,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "5,", "select (col2 between 3 and 1) is null from testbetweenasymmetric", new String[]{"t"}); + + testEval(schema, "testbetweenasymmetric", "0,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "1,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "2,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "3,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); + testEval(schema, "testbetweenasymmetric", "4,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "5,", "select (col2 not between 1 and 3) is null from testbetweenasymmetric", new String[]{"t"}); + + testEval(schema, "testbetweenasymmetric", "0,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "1,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "2,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "3,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "4,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "testbetweenasymmetric", "5,", "select (col2 not between 3 and 1) is null from testbetweenasymmetric", new String[]{"t"}); } @Test @@ -198,33 +198,33 @@ public void testBetweenSymmetric() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TajoDataTypes.Type.INT4); schema.addColumn("col2", TajoDataTypes.Type.INT4); - testEval(schema, "table1", "0,", "select col1 between symmetric 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "1,", "select col1 between symmetric 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "2,", "select col1 between symmetric 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "3,", "select col1 between symmetric 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "4,", "select col1 between symmetric 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "5,", "select (col2 between symmetric 1 and 3) is null from table1", new String[]{"t"}); - - testEval(schema, "table1", "0,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "1,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "2,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "3,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"f"}); - testEval(schema, "table1", "4,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"t"}); - testEval(schema, "table1", "5,", "select (col2 not between symmetric 1 and 3) is null from table1", new String[]{"t"}); - - testEval(schema, "table1", "0,", "select col1 between symmetric 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "1,", "select col1 between symmetric 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "2,", "select col1 between symmetric 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "3,", "select col1 between symmetric 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "4,", "select col1 between symmetric 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "5,", "select (col2 between symmetric 3 and 1) is null from table1", new String[]{"t"}); - - testEval(schema, "table1", "0,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "1,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "2,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "3,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"f"}); - testEval(schema, "table1", "4,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"t"}); - testEval(schema, "table1", "5,", "select (col2 not between symmetric 3 and 1) is null from table1", + testEval(schema, "testbetweensymmetric", "0,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "1,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "2,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "3,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "4,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "5,", "select (col2 between symmetric 1 and 3) is null from testbetweensymmetric", new String[]{"t"}); + + testEval(schema, "testbetweensymmetric", "0,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "1,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "2,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "3,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "4,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "5,", "select (col2 not between symmetric 1 and 3) is null from testbetweensymmetric", new String[]{"t"}); + + testEval(schema, "testbetweensymmetric", "0,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "1,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "2,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "3,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "4,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "5,", "select (col2 between symmetric 3 and 1) is null from testbetweensymmetric", new String[]{"t"}); + + testEval(schema, "testbetweensymmetric", "0,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "1,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "2,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "3,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); + testEval(schema, "testbetweensymmetric", "4,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); + testEval(schema, "testbetweensymmetric", "5,", "select (col2 not between symmetric 3 and 1) is null from testbetweensymmetric", new String[]{"t"}); } @@ -243,56 +243,56 @@ public void testUnary() throws IOException { // sign test - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from table1;", new String [] {"5.1"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from table1;", new String [] {"-1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from table1;", new String [] {"-2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from table1;", new String [] {"-3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from table1;", new String [] {"-4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from table1;", new String [] {"-5.1"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from testunary;", new String [] {"1"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from testunary;", new String [] {"2"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from testunary;", new String [] {"3"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from testunary;", new String [] {"4.1"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from testunary;", new String [] {"5.1"}); + + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from testunary;", new String [] {"-1"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from testunary;", new String [] {"-2"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from testunary;", new String [] {"-3"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from testunary;", new String [] {"-4.1"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from testunary;", new String [] {"-5.1"}); // not test - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select col8 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select NOT (col8) from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select NOT(NOT (col8)) from table1;", new String [] {"t"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select col8 from testunary;", new String [] {"t"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select NOT (col8) from testunary;", new String [] {"f"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select NOT(NOT (col8)) from testunary;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,", "select col8 is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,", "select (NOT (col8)) is null from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,", "select (NOT(NOT (col8))) is null from table1;", new String [] {"t"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,", "select col8 is null from testunary;", new String [] {"t"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,", "select (NOT (col8)) is null from testunary;", new String [] {"t"}); + testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,", "select (NOT(NOT (col8))) is null from testunary;", new String [] {"t"}); } @Test public void testAndOr() throws IOException { testSimpleEval("select true or (false or false) or false;", new String[] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true and true;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true and false;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false and true;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false and false;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true and true;", new String [] {"t"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true and false;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false and true;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false and false;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true or true;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true or false;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false or true;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false or false;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true or true;", new String [] {"t"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true or false;", new String [] {"t"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false or true;", new String [] {"t"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false or false;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (true and true) and false;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (true and false) and true;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (false and true) and true;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (true and true) and false;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (true and false) and true;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (false and true) and true;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (1 < 2) and true;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (1 < 2) and false;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (1 < 2) or false;", new String [] {"t"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (1 < 2) and true;", new String [] {"t"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (1 < 2) and false;", new String [] {"f"}); + testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (1 < 2) or false;", new String [] {"t"}); } @Test public void testFunction() throws IOException { - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('abc');", new String [] {"ABC"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('bbc');", new String [] {"BBC"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('chs');", new String [] {"CHS"}); + testEval(schema, "testfunction", "0,1,2,3,4.5,6.5", "select upper('abc');", new String [] {"ABC"}); + testEval(schema, "testfunction", "0,1,2,3,4.5,6.5", "select upper('bbc');", new String [] {"BBC"}); + testEval(schema, "testfunction", "0,1,2,3,4.5,6.5", "select upper('chs');", new String [] {"CHS"}); testSimpleEval("select ltrim('xxtrim', 'xx') ", new String[]{"trim"}); } @@ -301,11 +301,11 @@ public void testFunction() throws IOException { public void testStringConcat() throws IOException { testSimpleEval("select length('123456') as col1 ", new String[]{"6"}); - testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 'abc' || 'bbc'", new String [] {"abcbbc"}); + testEval(schema, "teststringconcat", "0,1,2,3,4.5,6.5", "select 'abc' || 'bbc'", new String [] {"abcbbc"}); Schema schema = new Schema(); schema.addColumn("col1", TajoDataTypes.Type.TEXT); schema.addColumn("col2", TajoDataTypes.Type.TEXT); - testEval(schema, "table1", " trim, abc", "select ltrim(col1) || ltrim(col2) from table1", + testEval(schema, "teststringconcat", " trim, abc", "select ltrim(col1) || ltrim(col2) from teststringconcat", new String[]{"trimabc"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java index 94d5e712e8..59d441c1a7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java @@ -93,8 +93,8 @@ public void testNot() throws IOException { schema1.addColumn("col3", INT4); testEval(schema1, - "table1", "123,123,456,-123", - "select col1 = col2, col1 = col3 from table1", + "testnot", "123,123,456,-123", + "select col1 = col2, col1 = col3 from testnot", new String[]{"t", "f"}); } @@ -132,12 +132,12 @@ public void testComparisonEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "table1", "123,123,456,-123", - "select col1 = col2, col1 = col3, col1 = col4 from table1", + "testcomparisonequal", "123,123,456,-123", + "select col1 = col2, col1 = col3, col1 = col4 from testcomparisonequal", new String[]{"t", "f", "f"}); testEval(schema1, - "table1", "123,123,,", - "select col1 = col2, (col1 = col3) is null, (col3 = col2) is null from table1", + "testcomparisonequal", "123,123,,", + "select col1 = col2, (col1 = col3) is null, (col3 = col2) is null from testcomparisonequal", new String[]{"t", "t", "t"}); } @@ -149,12 +149,12 @@ public void testComparisonNotEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "table1", "123,123,456,-123", - "select col1 <> col2, col1 <> col3, col1 <> col4 from table1", + "testcomparisonnotequal", "123,123,456,-123", + "select col1 <> col2, col1 <> col3, col1 <> col4 from testcomparisonnotequal", new String[]{"f", "t", "t"}); testEval(schema1, - "table1", "123,123,,", - "select col1 <> col2, (col1 <> col3) is null, (col3 <> col2) is null from table1", + "testcomparisonnotequal", "123,123,,", + "select col1 <> col2, (col1 <> col3) is null, (col3 <> col2) is null from testcomparisonnotequal", new String[]{"f", "t", "t"}); } @@ -166,12 +166,12 @@ public void testComparisonLessThan() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "table1", "123,123,456,-123", - "select col1 < col2, col1 < col3, col1 < col4 from table1", + "testcomparisonlessthan", "123,123,456,-123", + "select col1 < col2, col1 < col3, col1 < col4 from testcomparisonlessthan", new String[]{"f", "t", "f"}); testEval(schema1, - "table1", "123,456,,", - "select col1 < col2, (col1 = col3) is null, (col4 = col1) is null from table1", + "testcomparisonlessthan", "123,456,,", + "select col1 < col2, (col1 = col3) is null, (col4 = col1) is null from testcomparisonlessthan", new String[]{"t", "t", "t"}); } @@ -183,12 +183,12 @@ public void testComparisonLessThanEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "table1", "123,123,456,-123", - "select col1 <= col2, col1 <= col3, col1 <= col4 from table1", + "testcomparisonlessthanequal", "123,123,456,-123", + "select col1 <= col2, col1 <= col3, col1 <= col4 from testcomparisonlessthanequal", new String[]{"t", "t", "f"}); testEval(schema1, - "table1", "123,456,,", - "select col1 <= col2, (col1 <= col3) is null, (col4 <= col1) is null from table1", + "testcomparisonlessthanequal", "123,456,,", + "select col1 <= col2, (col1 <= col3) is null, (col4 <= col1) is null from testcomparisonlessthanequal", new String[]{"t", "t", "t"}); } @@ -200,12 +200,12 @@ public void testComparisonGreaterThan() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "table1", "123,123,456,-123", - "select col1 > col2, col3 > col2, col1 > col4 from table1", + "testcomparisongreaterthan", "123,123,456,-123", + "select col1 > col2, col3 > col2, col1 > col4 from testcomparisongreaterthan", new String[]{"f", "t", "t"}); testEval(schema1, - "table1", "123,456,,", - "select col2 > col1, col1 > col2, (col1 > col3) is null, (col4 > col1) is null from table1", + "testcomparisongreaterthan", "123,456,,", + "select col2 > col1, col1 > col2, (col1 > col3) is null, (col4 > col1) is null from testcomparisongreaterthan", new String[]{"t", "f", "t", "t"}); } @@ -217,12 +217,12 @@ public void testComparisonGreaterThanEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "table1", "123,123,456,-123", - "select col1 >= col2, col3 >= col2, col1 >= col4 from table1", + "testcomparisongreaterthanequal", "123,123,456,-123", + "select col1 >= col2, col3 >= col2, col1 >= col4 from testcomparisongreaterthanequal", new String[]{"t", "t", "t"}); testEval(schema1, - "table1", "123,456,,", - "select col2 >= col1, col1 >= col2, (col1 >= col3) is null, (col4 >= col1) is null from table1", + "testcomparisongreaterthanequal", "123,456,,", + "select col2 >= col1, col1 >= col2, (col1 >= col3) is null, (col4 >= col1) is null from testcomparisongreaterthanequal", new String[]{"t", "f", "t", "t"}); } @@ -238,20 +238,20 @@ public void testBetween() throws IOException { schema2.addColumn("col3", TEXT); // constant checker - testEval(schema2, "table1", "b,a,c", "select col1 between 'a' and 'c' from table1", new String[]{"t"}); - testEval(schema2, "table1", "b,a,c", "select col1 between 'c' and 'a' from table1", new String[]{"f"}); - testEval(schema2, "table1", "b,a,c", "select col1 between symmetric 'c' and 'a' from table1", new String[]{"t"}); - testEval(schema2, "table1", "d,a,c", "select col1 between 'a' and 'c' from table1", new String[]{"f"}); + testEval(schema2, "testbetween", "b,a,c", "select col1 between 'a' and 'c' from testbetween", new String[]{"t"}); + testEval(schema2, "testbetween", "b,a,c", "select col1 between 'c' and 'a' from testbetween", new String[]{"f"}); + testEval(schema2, "testbetween", "b,a,c", "select col1 between symmetric 'c' and 'a' from testbetween", new String[]{"t"}); + testEval(schema2, "testbetween", "d,a,c", "select col1 between 'a' and 'c' from testbetween", new String[]{"f"}); // tests for inclusive - testEval(schema2, "table1", "a,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"}); - testEval(schema2, "table1", "b,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"}); - testEval(schema2, "table1", "c,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"}); - testEval(schema2, "table1", "d,a,c", "select col1 between col2 and col3 from table1", new String[]{"f"}); + testEval(schema2, "testbetween", "a,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"t"}); + testEval(schema2, "testbetween", "b,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"t"}); + testEval(schema2, "testbetween", "c,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"t"}); + testEval(schema2, "testbetween", "d,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"f"}); // tests for asymmetric and symmetric - testEval(schema2, "table1", "b,a,c", "select col1 between col3 and col2 from table1", new String[]{"f"}); - testEval(schema2, "table1", "b,a,c", "select col1 between symmetric col3 and col2 from table1", new String[]{"t"}); + testEval(schema2, "testbetween", "b,a,c", "select col1 between col3 and col2 from testbetween", new String[]{"f"}); + testEval(schema2, "testbetween", "b,a,c", "select col1 between symmetric col3 and col2 from testbetween", new String[]{"t"}); } @Test @@ -266,27 +266,27 @@ public void testBetween2() throws IOException { // for TAJO-249 "case " + "when date_a BETWEEN 20130705 AND 20130715 AND ((date_b BETWEEN 20100101 AND 20120601) OR date_b > 20130715) " + "AND (date_c < 20120601 OR date_c > 20130715) AND date_d > 20130715" + - "then 1 else 0 end from table1"; + "then 1 else 0 end from testbetween2"; - testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "table1", "20130716,20100102,20120525,20130716", query, new String [] {"0"}); + testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "testbetween2", "20130716,20100102,20120525,20130716", query, new String [] {"0"}); // date_b - testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "table1", "20130715,20120602,20120525,20130716", query, new String [] {"0"}); - testEval(schema3, "table1", "20130715,20091201,20120525,20130716", query, new String [] {"0"}); - testEval(schema3, "table1", "20130715,20130716,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "testbetween2", "20130715,20120602,20120525,20130716", query, new String [] {"0"}); + testEval(schema3, "testbetween2", "20130715,20091201,20120525,20130716", query, new String [] {"0"}); + testEval(schema3, "testbetween2", "20130715,20130716,20120525,20130716", query, new String [] {"1"}); // date_c - testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "table1", "20130715,20100102,20120602,20130716", query, new String [] {"0"}); + testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "testbetween2", "20130715,20100102,20120602,20130716", query, new String [] {"0"}); - testEval(schema3, "table1", "20130715,20100102,20130716,20130716", query, new String [] {"1"}); - testEval(schema3, "table1", "20130715,20100102,20130714,20130716", query, new String [] {"0"}); + testEval(schema3, "testbetween2", "20130715,20100102,20130716,20130716", query, new String [] {"1"}); + testEval(schema3, "testbetween2", "20130715,20100102,20130714,20130716", query, new String [] {"0"}); // date_d - testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "table1", "20130715,20100102,20120525,20130705", query, new String [] {"0"}); + testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130705", query, new String [] {"0"}); } ////////////////////////////////////////////////////////////////// @@ -300,21 +300,21 @@ public void testInPredicateWithConstant() throws IOException { schema2.addColumn("col2", TEXT); schema2.addColumn("col3", TEXT); - testEval(schema2, "table1", "a,b,c", "select col1 in ('a'), col2 in ('a', 'c') from table1", new String[]{"t","f"}); - testEval(schema2, "table1", "a,,c", "select col1 in ('a','b','c'), (col2 in ('a', 'c')) is null from table1", + testEval(schema2, "testinpredicatewithconstant", "a,b,c", "select col1 in ('a'), col2 in ('a', 'c') from testinpredicatewithconstant", new String[]{"t","f"}); + testEval(schema2, "testinpredicatewithconstant", "a,,c", "select col1 in ('a','b','c'), (col2 in ('a', 'c')) is null from testinpredicatewithconstant", new String[]{"t","t"}); testEval(schema2, - "table1", + "testinpredicatewithconstant", "2014-03-21,2015-04-01,2016-04-01", - "select substr(col1,1,4) in ('2014','2015','2016'), substr(col1,6,2)::int4 in (1,2,3) from table1", + "select substr(col1,1,4) in ('2014','2015','2016'), substr(col1,6,2)::int4 in (1,2,3) from testinpredicatewithconstant", new String[]{"t", "t"}); // null handling test testEval(schema2, - "table1", + "testinpredicatewithconstant", "2014-03-21,,2015-04-01", - "select (substr(col2,1,4)::int4 in (2014,2015,2016)) is null from table1", + "select (substr(col2,1,4)::int4 in (2014,2015,2016)) is null from testinpredicatewithconstant", new String[]{"t"}); } @@ -325,16 +325,16 @@ public void testInPredicateWithSimpleExprs() throws IOException { schema2.addColumn("col2", INT4); schema2.addColumn("col3", TEXT); - testEval(schema2, "table1", "abc,2,3", "select col1 in ('a'||'b'||'c'), col2 in (1 + 1, 2 * 10, 2003) from table1", + testEval(schema2, "testinpredicatewithsimpleexprs", "abc,2,3", "select col1 in ('a'||'b'||'c'), col2 in (1 + 1, 2 * 10, 2003) from testinpredicatewithsimpleexprs", new String[]{"t","t"}); - testEval(schema2, "table1", "abc,2,3", "select col1 in ('a'||'b'), col2 in ('1'::int, '2'::int, 3) from table1", + testEval(schema2, "testinpredicatewithsimpleexprs", "abc,2,3", "select col1 in ('a'||'b'), col2 in ('1'::int, '2'::int, 3) from testinpredicatewithsimpleexprs", new String[]{"f","t"}); testEval(schema2, - "table1", + "testinpredicatewithsimpleexprs", "abc,,3", - "select col1 in (reverse('cba')), (col2 in ('1'::int, '2'::int, 3)) is null from table1", + "select col1 in (reverse('cba')), (col2 in ('1'::int, '2'::int, 3)) is null from testinpredicatewithsimpleexprs", new String[]{"t","t"}); } @@ -347,9 +347,9 @@ public void testIsNullPredicate() throws IOException { Schema schema1 = new Schema(); schema1.addColumn("col1", INT4); schema1.addColumn("col2", INT4); - testEval(schema1, "table1", "123,", "select col1 is null, col2 is null as a from table1", + testEval(schema1, "testisnullpredicate", "123,", "select col1 is null, col2 is null as a from testisnullpredicate", new String[]{"f", "t"}); - testEval(schema1, "table1", "123,", "select col1 is not null, col2 is not null as a from table1", + testEval(schema1, "testisnullpredicate", "123,", "select col1 is not null, col2 is not null as a from testisnullpredicate", new String[]{"t", "f"}); } @@ -358,11 +358,11 @@ public void testIsNullPredicateWithFunction() throws IOException { Schema schema2 = new Schema(); schema2.addColumn("col1", TEXT); schema2.addColumn("col2", TEXT); - testEval(schema2, "table1", "_123,", "select ltrim(col1, '_') is null, upper(col2) is null as a from table1", + testEval(schema2, "testisnullpredicatewithfunction", "_123,", "select ltrim(col1, '_') is null, upper(col2) is null as a from testisnullpredicatewithfunction", new String[]{"f", "t"}); - testEval(schema2, "table1", "_123,", - "select ltrim(col1, '_') is not null, upper(col2) is not null as a from table1", new String[]{"t", "f"}); + testEval(schema2, "testisnullpredicatewithfunction", "_123,", + "select ltrim(col1, '_') is not null, upper(col2) is not null as a from testisnullpredicatewithfunction", new String[]{"t", "f"}); } ////////////////////////////////////////////////////////////////// @@ -397,10 +397,10 @@ public void testBooleanTestOnTable() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", BOOLEAN); schema.addColumn("col2", BOOLEAN); - testEval(schema, "table1", "t,f", "select col1 is true, col2 is false from table1", new String [] {"t", "t"}); - testEval(schema, "table1", "t,f", "select col1 is not true, col2 is not false from table1", + testEval(schema, "testbooleantestontable", "t,f", "select col1 is true, col2 is false from testbooleantestontable", new String [] {"t", "t"}); + testEval(schema, "testbooleantestontable", "t,f", "select col1 is not true, col2 is not false from testbooleantestontable", new String [] {"f", "f"}); - testEval(schema, "table1", "t,f", "select not col1 is not true, not col2 is not false from table1", + testEval(schema, "testbooleantestontable", "t,f", "select not col1 is not true, not col2 is not false from testbooleantestontable", new String [] {"t", "t"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java index b04d544862..20a29c1184 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java @@ -106,69 +106,69 @@ public void testExplicitCast() throws IOException { testSimpleEval("select cast (1 as char)", new String[]{"1"}); testSimpleEval("select cast (119 as char)", new String[] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int1 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int1 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int1 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int1 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int1 from table1;", new String [] {"5"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int1 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int1 from table1;", new String [] {"7"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int2 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int2 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int2 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int2 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int2 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int2 from table1;", new String [] {"5"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int2 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int2 from table1;", new String [] {"7"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int4 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int4 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int4 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int4 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int4 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int4 from table1;", new String [] {"5"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int4 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int4 from table1;", new String [] {"7"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int8 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int8 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int8 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int8 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int8 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int8 from table1;", new String [] {"5"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int8 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int8 from table1;", new String [] {"7"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::float4 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::float4 from table1;", new String [] {"1.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::float4 from table1;", new String [] {"2.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::float4 from table1;", new String [] {"3.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::float4 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::float4 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::float4 from table1;", new String [] {"6.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::float4 from table1;", new String [] {"7.0"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::float8 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::float8 from table1;", new String [] {"1.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::float8 from table1;", new String [] {"2.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::float8 from table1;", new String [] {"3.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::float8 from table1;", new String[] + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int1 from testexplicitcast;", new String [] {"0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int1 from testexplicitcast;", new String [] {"1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int1 from testexplicitcast;", new String [] {"2"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int1 from testexplicitcast;", new String [] {"3"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int1 from testexplicitcast;", new String [] {"4"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int1 from testexplicitcast;", new String [] {"5"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int1 from testexplicitcast;", new String [] {"6"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int1 from testexplicitcast;", new String [] {"7"}); + + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int2 from testexplicitcast;", new String [] {"0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int2 from testexplicitcast;", new String [] {"1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int2 from testexplicitcast;", new String [] {"2"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int2 from testexplicitcast;", new String [] {"3"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int2 from testexplicitcast;", new String [] {"4"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int2 from testexplicitcast;", new String [] {"5"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int2 from testexplicitcast;", new String [] {"6"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int2 from testexplicitcast;", new String [] {"7"}); + + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int4 from testexplicitcast;", new String [] {"0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int4 from testexplicitcast;", new String [] {"1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int4 from testexplicitcast;", new String [] {"2"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int4 from testexplicitcast;", new String [] {"3"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int4 from testexplicitcast;", new String [] {"4"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int4 from testexplicitcast;", new String [] {"5"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int4 from testexplicitcast;", new String [] {"6"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int4 from testexplicitcast;", new String [] {"7"}); + + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int8 from testexplicitcast;", new String [] {"0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int8 from testexplicitcast;", new String [] {"1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int8 from testexplicitcast;", new String [] {"2"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int8 from testexplicitcast;", new String [] {"3"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int8 from testexplicitcast;", new String [] {"4"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int8 from testexplicitcast;", new String [] {"5"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int8 from testexplicitcast;", new String [] {"6"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int8 from testexplicitcast;", new String [] {"7"}); + + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::float4 from testexplicitcast;", new String [] {"0.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::float4 from testexplicitcast;", new String [] {"1.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::float4 from testexplicitcast;", new String [] {"2.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::float4 from testexplicitcast;", new String [] {"3.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::float4 from testexplicitcast;", new String [] {"4.1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::float4 from testexplicitcast;", new String [] {"5.1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::float4 from testexplicitcast;", new String [] {"6.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::float4 from testexplicitcast;", new String [] {"7.0"}); + + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::float8 from testexplicitcast;", new String [] {"0.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::float8 from testexplicitcast;", new String [] {"1.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::float8 from testexplicitcast;", new String [] {"2.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::float8 from testexplicitcast;", new String [] {"3.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::float8 from testexplicitcast;", new String[] {Double.valueOf(4.1f).toString()}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::float8 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::float8 from table1;", new String [] {"6.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::float8 from table1;", new String [] {"7.0"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::text from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::text from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::text from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::text from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::text from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::text from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::text from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::text from table1;", new String [] {"7"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::float8 from testexplicitcast;", new String [] {"5.1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::float8 from testexplicitcast;", new String [] {"6.0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::float8 from testexplicitcast;", new String [] {"7.0"}); + + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::text from testexplicitcast;", new String [] {"0"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::text from testexplicitcast;", new String [] {"1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::text from testexplicitcast;", new String [] {"2"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::text from testexplicitcast;", new String [] {"3"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::text from testexplicitcast;", new String [] {"4.1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::text from testexplicitcast;", new String [] {"5.1"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::text from testexplicitcast;", new String [] {"6"}); + testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::text from testexplicitcast;", new String [] {"7"}); } @Test @@ -183,94 +183,94 @@ public void testImplicitCastForInt1() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col0 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col1 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col2 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col3 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col4 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col5 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col6::int1 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col7::int1 from table1;", new String [] {"7"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col0 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col1 from table1;", new String [] {"-1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col2 from table1;", new String [] {"-2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col3 from table1;", new String [] {"-3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col4 from table1;", new String [] {"-4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col5 from table1;", new String [] {"-5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col6::int1 from table1;", new String [] {"-6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col7::int1 from table1;", new String [] {"-7"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col0 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col2 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col3 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col4 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col5 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col6::int1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col7::int1 from table1;", new String [] {"0"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col2 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col3 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col4 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col5 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col6::int1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col7::int1 from table1;", new String [] {"0"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col6::int1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col7::int1 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col6::int1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col7::int1 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col6::int1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col7::int1 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col6::int1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col7::int1 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col6::int1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col7::int1 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col6::int1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col7::int1 from table1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col0 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col1 from testimplicitcastforint1;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col2 from testimplicitcastforint1;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col3 from testimplicitcastforint1;", new String [] {"3"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col4 from testimplicitcastforint1;", new String [] {"4.1"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col5 from testimplicitcastforint1;", new String [] {"5.1"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col6::int1 from testimplicitcastforint1;", new String [] {"6"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col7::int1 from testimplicitcastforint1;", new String [] {"7"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col0 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col1 from testimplicitcastforint1;", new String [] {"-1"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col2 from testimplicitcastforint1;", new String [] {"-2"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col3 from testimplicitcastforint1;", new String [] {"-3"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col4 from testimplicitcastforint1;", new String [] {"-4.1"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col5 from testimplicitcastforint1;", new String [] {"-5.1"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col6::int1 from testimplicitcastforint1;", new String [] {"-6"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col7::int1 from testimplicitcastforint1;", new String [] {"-7"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col0 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col1 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col2 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col3 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col4 from testimplicitcastforint1;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col5 from testimplicitcastforint1;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col6::int1 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col7::int1 from testimplicitcastforint1;", new String [] {"0"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col1 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col2 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col3 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col4 from testimplicitcastforint1;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col5 from testimplicitcastforint1;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col6::int1 from testimplicitcastforint1;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col7::int1 from testimplicitcastforint1;", new String [] {"0"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col0 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col1 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col2 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col3 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col4 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col5 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col6::int1 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col7::int1 from testimplicitcastforint1;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col0 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col1 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col2 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col3 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col4 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col5 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col6::int1 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col7::int1 from testimplicitcastforint1;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col0 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col1 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col2 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col3 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col4 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col5 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col6::int1 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col7::int1 from testimplicitcastforint1;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col0 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col1 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col2 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col3 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col4 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col5 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col6::int1 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col7::int1 from testimplicitcastforint1;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col0 from testimplicitcastforint1;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col1 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col2 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col3 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col4 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col5 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col6::int1 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col7::int1 from testimplicitcastforint1;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col0 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col1 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col2 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col3 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col4 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col5 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col6::int1 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col7::int1 from testimplicitcastforint1;", new String [] {"t"}); } @Test @@ -285,94 +285,94 @@ public void testImplicitCastForInt2() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col0 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col1 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col2 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col3 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col4 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col5 from table1;", new String [] {"6.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col6::int2 from table1;", new String [] {"7"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col7::int2 from table1;", new String [] {"8"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col0 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col2 from table1;", new String [] {"-1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col3 from table1;", new String [] {"-2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col4 from table1;", new String [] {"-3.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col5 from table1;", new String [] {"-4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col6::int2 from table1;", new String [] {"-5"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col7::int2 from table1;", new String [] {"-6"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col0 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col1 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col2 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col3 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col4 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col5 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col6::int2 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col7::int2 from table1;", new String [] {"7"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col2 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col3 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col4 from table1;", new String [] {"1.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col5 from table1;", new String [] {"1.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col6::int2 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col7::int2 from table1;", new String [] {"1"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col6::int2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col7::int2 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col6::int2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col7::int2 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col6::int2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col7::int2 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col6::int2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col7::int2 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col6::int2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col7::int2 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col6::int2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col7::int2 from table1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col0 from testimplicitcastforint2;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col1 from testimplicitcastforint2;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col2 from testimplicitcastforint2;", new String [] {"3"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col3 from testimplicitcastforint2;", new String [] {"4"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col4 from testimplicitcastforint2;", new String [] {"5.1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col5 from testimplicitcastforint2;", new String [] {"6.1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col6::int2 from testimplicitcastforint2;", new String [] {"7"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col7::int2 from testimplicitcastforint2;", new String [] {"8"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col0 from testimplicitcastforint2;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col1 from testimplicitcastforint2;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col2 from testimplicitcastforint2;", new String [] {"-1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col3 from testimplicitcastforint2;", new String [] {"-2"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col4 from testimplicitcastforint2;", new String [] {"-3.1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col5 from testimplicitcastforint2;", new String [] {"-4.1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col6::int2 from testimplicitcastforint2;", new String [] {"-5"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col7::int2 from testimplicitcastforint2;", new String [] {"-6"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col0 from testimplicitcastforint2;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col1 from testimplicitcastforint2;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col2 from testimplicitcastforint2;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col3 from testimplicitcastforint2;", new String [] {"3"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col4 from testimplicitcastforint2;", new String [] {"4.1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col5 from testimplicitcastforint2;", new String [] {"5.1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col6::int2 from testimplicitcastforint2;", new String [] {"6"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col7::int2 from testimplicitcastforint2;", new String [] {"7"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col1 from testimplicitcastforint2;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col2 from testimplicitcastforint2;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col3 from testimplicitcastforint2;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col4 from testimplicitcastforint2;", new String [] {"1.0"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col5 from testimplicitcastforint2;", new String [] {"1.0"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col6::int2 from testimplicitcastforint2;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col7::int2 from testimplicitcastforint2;", new String [] {"1"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col0 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col1 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col2 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col3 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col4 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col5 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col6::int2 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col7::int2 from testimplicitcastforint2;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col0 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col1 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col2 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col3 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col4 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col5 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col6::int2 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col7::int2 from testimplicitcastforint2;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col0 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col1 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col2 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col3 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col4 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col5 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col6::int2 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col7::int2 from testimplicitcastforint2;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col0 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col1 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col2 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col3 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col4 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col5 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col6::int2 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col7::int2 from testimplicitcastforint2;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col0 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col1 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col2 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col3 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col4 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col5 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col6::int2 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col7::int2 from testimplicitcastforint2;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col0 from testimplicitcastforint2;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col1 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col2 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col3 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col4 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col5 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col6::int2 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col7::int2 from testimplicitcastforint2;", new String [] {"t"}); } @Test @@ -387,95 +387,95 @@ public void testImplicitCastForInt4() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col0 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col1 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col2 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col3 from table1;", new String [] {"5"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col4 from table1;", new String [] {"6.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col5 from table1;", new String [] {"7.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col6::int4 from table1;", new String [] {"8"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col7::int4 from table1;", new String [] {"9"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col0 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col1 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col2 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col3 from table1;", new String [] {"-1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col4 from table1;", new String [] {"-2.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col0 from testimplicitcastforint4;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col1 from testimplicitcastforint4;", new String [] {"3"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col2 from testimplicitcastforint4;", new String [] {"4"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col3 from testimplicitcastforint4;", new String [] {"5"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col4 from testimplicitcastforint4;", new String [] {"6.1"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col5 from testimplicitcastforint4;", new String [] {"7.1"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col6::int4 from testimplicitcastforint4;", new String [] {"8"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col7::int4 from testimplicitcastforint4;", new String [] {"9"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col0 from testimplicitcastforint4;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col1 from testimplicitcastforint4;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col2 from testimplicitcastforint4;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col3 from testimplicitcastforint4;", new String [] {"-1"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col4 from testimplicitcastforint4;", new String [] {"-2.1"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col5 from testimplicitcastforint4;", new String [] { (new Integer(2) - 5.1d) +""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col6::int4 from table1;", new String [] {"-4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col7::int4 from table1;", new String [] {"-5"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col0 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col1 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col2 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col3 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col4 from table1;", new String [] {"8.2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col5 from table1;", new String [] {"10.2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col6::int4 from table1;", new String [] {"12"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col7::int4 from table1;", new String [] {"14"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col2 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col3 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col4 from table1;", new String [] {"2.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col5 from table1;", new String [] {"2.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col6::int4 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col7::int4 from table1;", new String [] {"2"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col6::int4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col7::int4 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col6::int4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col7::int4 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col6::int4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col7::int4 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col6::int4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col7::int4 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col6::int4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col7::int4 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col6::int4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col7::int4 from table1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col6::int4 from testimplicitcastforint4;", new String [] {"-4"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col7::int4 from testimplicitcastforint4;", new String [] {"-5"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col0 from testimplicitcastforint4;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col1 from testimplicitcastforint4;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col2 from testimplicitcastforint4;", new String [] {"4"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col3 from testimplicitcastforint4;", new String [] {"6"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col4 from testimplicitcastforint4;", new String [] {"8.2"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col5 from testimplicitcastforint4;", new String [] {"10.2"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col6::int4 from testimplicitcastforint4;", new String [] {"12"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col7::int4 from testimplicitcastforint4;", new String [] {"14"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col1 from testimplicitcastforint4;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col2 from testimplicitcastforint4;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col3 from testimplicitcastforint4;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col4 from testimplicitcastforint4;", new String [] {"2.0"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col5 from testimplicitcastforint4;", new String [] {"2.0"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col6::int4 from testimplicitcastforint4;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col7::int4 from testimplicitcastforint4;", new String [] {"2"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col0 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col1 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col2 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col3 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col4 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col5 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col6::int4 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col7::int4 from testimplicitcastforint4;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col0 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col1 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col2 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col3 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col4 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col5 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col6::int4 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col7::int4 from testimplicitcastforint4;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col0 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col1 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col2 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col3 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col4 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col5 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col6::int4 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col7::int4 from testimplicitcastforint4;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col0 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col1 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col2 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col3 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col4 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col5 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col6::int4 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col7::int4 from testimplicitcastforint4;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col0 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col1 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col2 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col3 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col4 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col5 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col6::int4 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col7::int4 from testimplicitcastforint4;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col0 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col1 from testimplicitcastforint4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col2 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col3 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col4 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col5 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col6::int4 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col7::int4 from testimplicitcastforint4;", new String [] {"t"}); } @Test @@ -490,99 +490,99 @@ public void testImplicitCastForInt8() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col0 from table1;", new String[]{"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col1 from table1;", new String [] {"4"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col2 from table1;", new String [] {"5"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col3 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col0 from testimplicitcastforint8;", new String[]{"3"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col1 from testimplicitcastforint8;", new String [] {"4"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col2 from testimplicitcastforint8;", new String [] {"5"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col3 from testimplicitcastforint8;", new String [] {"6"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col4 from testimplicitcastforint8;", new String [] { (new Long(3) + new Float(4.1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col5 from table1;", new String [] {"8.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col6::int8 from table1;", new String [] {"9"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col7::int8 from table1;", new String [] {"10"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col0 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col1 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col2 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col3 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col5 from testimplicitcastforint8;", new String [] {"8.1"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col6::int8 from testimplicitcastforint8;", new String [] {"9"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col7::int8 from testimplicitcastforint8;", new String [] {"10"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col0 from testimplicitcastforint8;", new String [] {"3"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col1 from testimplicitcastforint8;", new String [] {"2"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col2 from testimplicitcastforint8;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col3 from testimplicitcastforint8;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col4 from testimplicitcastforint8;", new String [] { (new Long(3) - new Float(4.1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col5 from testimplicitcastforint8;", new String [] { (new Long(3) - 5.1d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col6::int8 from table1;", new String [] {"-3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col7::int8 from table1;", new String [] {"-4"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col0 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col1 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col2 from table1;", new String [] {"6"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col3 from table1;", new String [] {"9"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col6::int8 from testimplicitcastforint8;", new String [] {"-3"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col7::int8 from testimplicitcastforint8;", new String [] {"-4"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col0 from testimplicitcastforint8;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col1 from testimplicitcastforint8;", new String [] {"3"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col2 from testimplicitcastforint8;", new String [] {"6"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col3 from testimplicitcastforint8;", new String [] {"9"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col4 from testimplicitcastforint8;", new String [] { (new Long(3) * new Float("4.1"))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col5 from testimplicitcastforint8;", new String [] { (new Long(3) * new Double("5.1"))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col6::int8 from table1;", new String [] {"18"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col7::int8 from table1;", new String [] {"21"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col1 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col2 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col3 from table1;", new String [] {"0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col4 from table1;", new String [] {"3.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col5 from table1;", new String [] {"3.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col6::int8 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col7::int8 from table1;", new String [] {"3"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col6::int8 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col7::int8 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col6::int8 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col7::int8 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col6::int8 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col7::int8 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col6::int8 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col7::int8 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col6::int8 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col7::int8 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col6::int8 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col7::int8 from table1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col6::int8 from testimplicitcastforint8;", new String [] {"18"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col7::int8 from testimplicitcastforint8;", new String [] {"21"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col1 from testimplicitcastforint8;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col2 from testimplicitcastforint8;", new String [] {"1"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col3 from testimplicitcastforint8;", new String [] {"0"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col4 from testimplicitcastforint8;", new String [] {"3.0"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col5 from testimplicitcastforint8;", new String [] {"3.0"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col6::int8 from testimplicitcastforint8;", new String [] {"3"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col7::int8 from testimplicitcastforint8;", new String [] {"3"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col0 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col1 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col2 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col3 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col4 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col5 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col6::int8 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col7::int8 from testimplicitcastforint8;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col0 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col1 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col2 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col3 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col4 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col5 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col6::int8 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col7::int8 from testimplicitcastforint8;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col0 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col1 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col2 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col3 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col4 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col5 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col6::int8 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col7::int8 from testimplicitcastforint8;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col0 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col1 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col2 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col3 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col4 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col5 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col6::int8 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col7::int8 from testimplicitcastforint8;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col0 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col1 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col2 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col3 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col4 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col5 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col6::int8 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col7::int8 from testimplicitcastforint8;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col0 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col1 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col2 from testimplicitcastforint8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col3 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col4 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col5 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col6::int8 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col7::int8 from testimplicitcastforint8;", new String [] {"t"}); } @Test @@ -597,111 +597,111 @@ public void testImplicitCastForFloat4() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col0 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col1 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col2 from table1;", new String [] {"6.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col3 from table1;", new String [] {"7.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col4 from table1;", new String [] {"8.2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col0 from testimplicitcastforfloat4;", new String [] {"4.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col1 from testimplicitcastforfloat4;", new String [] {"5.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col2 from testimplicitcastforfloat4;", new String [] {"6.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col3 from testimplicitcastforfloat4;", new String [] {"7.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col4 from testimplicitcastforfloat4;", new String [] {"8.2"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col5 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) + 5.1d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col6::float4 from table1;", new String [] {"10.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col7::float4 from table1;", new String [] {"11.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col6::float4 from testimplicitcastforfloat4;", new String [] {"10.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col7::float4 from testimplicitcastforfloat4;", new String [] {"11.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col0 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col1 from table1;", new String [] {"3.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col2 from table1;", new String [] {"2.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col3 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col0 from testimplicitcastforfloat4;", new String [] {"4.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col1 from testimplicitcastforfloat4;", new String [] {"3.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col2 from testimplicitcastforfloat4;", new String [] {"2.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col3 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) - new Long(3))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col4 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col4 from testimplicitcastforfloat4;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col5 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) - 5.1d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col6::float4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col6::float4 from testimplicitcastforfloat4;", new String [] { (4.1f - 6f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col7::float4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col7::float4 from testimplicitcastforfloat4;", new String [] { (4.1f - 7f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col0 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col1 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col2 from table1;", new String [] {"8.2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col3 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col0 from testimplicitcastforfloat4;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col1 from testimplicitcastforfloat4;", new String [] {"4.1"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col2 from testimplicitcastforfloat4;", new String [] {"8.2"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col3 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) * new Long(3))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col4 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) * new Float(4.1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col5 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) * 5.1d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col6::float4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col6::float4 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) * 6f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col7::float4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col7::float4 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) * 7f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col1 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col1 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) % new Integer(1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col2 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col2 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) % new Integer(2))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col3 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col3 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) % new Long(3))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col4 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) % new Float(4.1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col5 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) % 5.1d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col6::float4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col6::float4 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) % 6f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col7::int1 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col7::int1 from testimplicitcastforfloat4;", new String [] { (new Float(4.1) % 7f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col6::int1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col7::int1 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col6::int1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col7::int1 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col6::int1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col7::int1 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col6::int1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col7::int1 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col6::int1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col7::int1 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col6::int1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col7::int1 from table1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col0 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col1 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col2 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col3 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col4 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col5 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col6::int1 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col7::int1 from testimplicitcastforfloat4;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col0 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col1 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col2 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col3 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col4 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col5 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col6::int1 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col7::int1 from testimplicitcastforfloat4;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col0 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col1 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col2 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col3 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col4 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col5 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col6::int1 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col7::int1 from testimplicitcastforfloat4;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col0 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col1 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col2 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col3 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col4 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col5 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col6::int1 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col7::int1 from testimplicitcastforfloat4;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col0 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col1 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col2 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col3 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col4 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col5 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col6::int1 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col7::int1 from testimplicitcastforfloat4;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col0 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col1 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col2 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col3 from testimplicitcastforfloat4;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col4 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col5 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col6::int1 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col7::int1 from testimplicitcastforfloat4;", new String [] {"t"}); } @Test @@ -716,112 +716,112 @@ public void testImplicitCastForFloat8() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col0 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col1 from table1;", new String [] {"6.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col2 from table1;", new String [] {"7.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col3 from table1;", new String [] {"8.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col0 from testimplicitcastforfloat8;", new String [] {"5.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col1 from testimplicitcastforfloat8;", new String [] {"6.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col2 from testimplicitcastforfloat8;", new String [] {"7.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col3 from testimplicitcastforfloat8;", new String [] {"8.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col4 from testimplicitcastforfloat8;", new String [] { (5.1d + 4.1f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col5 from table1;", new String [] {"10.2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col6::int1 from table1;", new String [] {"11.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col7::int1 from table1;", new String [] {"12.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col5 from testimplicitcastforfloat8;", new String [] {"10.2"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col6::int1 from testimplicitcastforfloat8;", new String [] {"11.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col7::int1 from testimplicitcastforfloat8;", new String [] {"12.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col0 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col1 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col2 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col0 from testimplicitcastforfloat8;", new String [] {"5.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col1 from testimplicitcastforfloat8;", new String [] {"4.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col2 from testimplicitcastforfloat8;", new String [] { (5.1d - new Integer(2))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col3 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col3 from testimplicitcastforfloat8;", new String [] { (5.1d - 3l)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col4 from testimplicitcastforfloat8;", new String [] { (5.1d - 4.1f)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col5 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col6::float8 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col5 from testimplicitcastforfloat8;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col6::float8 from testimplicitcastforfloat8;", new String [] { (5.1d - 6d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col7::float8 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col7::float8 from testimplicitcastforfloat8;", new String [] { (5.1d - 7d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col0 from table1;", new String [] {"0.0"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col1 from table1;", new String [] {"5.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col2 from table1;", new String [] {"10.2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col3 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col0 from testimplicitcastforfloat8;", new String [] {"0.0"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col1 from testimplicitcastforfloat8;", new String [] {"5.1"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col2 from testimplicitcastforfloat8;", new String [] {"10.2"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col3 from testimplicitcastforfloat8;", new String [] { (5.1d * new Long(3))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col4 from testimplicitcastforfloat8;", new String [] { (5.1d * new Float(4.1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col5 from testimplicitcastforfloat8;", new String [] { (5.1d * 5.1d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col6::float8 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col6::float8 from testimplicitcastforfloat8;", new String [] { (5.1d * 6d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col7::float8 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col7::float8 from testimplicitcastforfloat8;", new String [] { (5.1d * 7d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col1 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col1 from testimplicitcastforfloat8;", new String [] { (5.1d % new Integer(1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col2 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col2 from testimplicitcastforfloat8;", new String [] { (5.1d % new Integer(2))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col3 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col3 from testimplicitcastforfloat8;", new String [] { (5.1d % new Long(3))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col4 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col4 from testimplicitcastforfloat8;", new String [] { (5.1d % new Float(4.1))+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col5 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col5 from testimplicitcastforfloat8;", new String [] { (5.1d % 5.1d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col6::float8 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col6::float8 from testimplicitcastforfloat8;", new String [] { (5.1d % 6d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col7::float8 from table1;", new String [] { + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col7::float8 from testimplicitcastforfloat8;", new String [] { (5.1d % 7d)+""}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col6::float8 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col7::float8 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col6::float8 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col7::float8 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col6::float8 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col7::float8 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col0 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col1 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col2 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col3 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col4 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col6::float8 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col7::float8 from table1;", new String [] {"f"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col5 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col6::float8 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col7::float8 from table1;", new String [] {"t"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col0 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col1 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col2 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col3 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col4 from table1;", new String [] {"f"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col5 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col6::float8 from table1;", new String [] {"t"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col7::float8 from table1;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col0 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col1 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col2 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col3 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col4 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col5 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col6::float8 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col7::float8 from testimplicitcastforfloat8;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col0 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col1 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col2 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col3 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col4 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col5 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col6::float8 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col7::float8 from testimplicitcastforfloat8;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col0 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col1 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col2 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col3 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col4 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col5 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col6::float8 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col7::float8 from testimplicitcastforfloat8;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col0 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col1 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col2 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col3 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col4 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col5 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col6::float8 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col7::float8 from testimplicitcastforfloat8;", new String [] {"f"}); + + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col0 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col1 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col2 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col3 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col4 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col5 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col6::float8 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col7::float8 from testimplicitcastforfloat8;", new String [] {"t"}); + + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col0 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col1 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col2 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col3 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col4 from testimplicitcastforfloat8;", new String [] {"f"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col5 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col6::float8 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col7::float8 from testimplicitcastforfloat8;", new String [] {"t"}); } @Test @@ -839,17 +839,17 @@ public void testSigned() throws IOException { // sign test - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from table1;", new String [] {"1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from table1;", new String [] {"2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from table1;", new String [] {"3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from table1;", new String [] {"4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from table1;", new String [] {"5.1"}); - - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from table1;", new String [] {"-1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from table1;", new String [] {"-2"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from table1;", new String [] {"-3"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from table1;", new String [] {"-4.1"}); - testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from table1;", new String [] {"-5.1"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from testsigned;", new String [] {"1"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from testsigned;", new String [] {"2"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from testsigned;", new String [] {"3"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from testsigned;", new String [] {"4.1"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from testsigned;", new String [] {"5.1"}); + + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from testsigned;", new String [] {"-1"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from testsigned;", new String [] {"-2"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from testsigned;", new String [] {"-3"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from testsigned;", new String [] {"-4.1"}); + testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from testsigned;", new String [] {"-5.1"}); } @Test @@ -875,18 +875,18 @@ public void testCastFromTable() throws IOException { schema.addColumn("col2", TEXT); testEval(queryContext, schema, - "table1", + "testcastfromtable", "123,234", - "select cast(col1 as float) as b, cast(col2 as float) as a from table1", + "select cast(col1 as float) as b, cast(col2 as float) as a from testcastfromtable", new String[]{"123.0", "234.0"}); - testEval(queryContext, schema, "table1", "123,234", "select col1::float, col2::float from table1", + testEval(queryContext, schema, "testcastfromtable", "123,234", "select col1::float, col2::float from testcastfromtable", new String[]{"123.0", "234.0"}); TimestampDatum timestamp = DatumFactory.createTimestamp("1980-04-01 01:50:01" + DateTimeUtil.getTimeZoneDisplayTime(tz)); - testEval(queryContext, schema, "table1", "1980-04-01 01:50:01,234", - "select col1::timestamp as t1, col2::float from table1 where t1 = '1980-04-01 01:50:01'::timestamp", + testEval(queryContext, schema, "testcastfromtable", "1980-04-01 01:50:01,234", + "select col1::timestamp as t1, col2::float from testcastfromtable where t1 = '1980-04-01 01:50:01'::timestamp", new String[]{TimestampDatum.asChars(timestamp.asTimeMeta(), tz, false), "234.0"} ); @@ -905,7 +905,7 @@ public void testBooleanLiteral() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); - testEval(schema, "table1", "123,234", "select col1, col2 from table1 where true", new String[]{"123", "234"}); + testEval(schema, "testbooleanliteral", "123,234", "select col1, col2 from testbooleanliteral where true", new String[]{"123", "234"}); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java index 72fdd6fa06..807ea6f78f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java @@ -57,10 +57,10 @@ public void testMaxLongWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("value", TajoDataTypes.Type.INT8); String[] data = new String[]{ "1|-111", "2|\\N", "3|-333" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testmaxlongwithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select max(value) as max_value from testbuiltin11"); + ResultSet res = executeString("select max(value) as max_value from testmaxlongwithnull"); String ascExpected = "max_value\n" + "-------------------------------\n" + "-111\n"; @@ -68,7 +68,7 @@ public void testMaxLongWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testmaxlongwithnull PURGE"); } } @@ -82,10 +82,10 @@ public void testMinMaxDate() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.DATE); String[] data = new String[]{ "2014-01-02", "2014-12-01", "2015-01-01", "1999-08-09", "2000-03-01" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminmaxdate", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxdate"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "1999-08-09,2015-01-01\n"; @@ -93,7 +93,7 @@ public void testMinMaxDate() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminmaxdate PURGE"); } } @@ -106,10 +106,10 @@ public void testMinMaxDateWithNull() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.DATE); String[] data = new String[]{ "2014-01-02", "2014-12-01", "\\N", "\\N", "2000-03-01" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminmaxdatewithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxdatewithnull"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "2000-03-01,2014-12-01\n"; @@ -117,7 +117,7 @@ public void testMinMaxDateWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminmaxdatewithnull PURGE"); } } @@ -130,10 +130,10 @@ public void testMinMaxTime() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.TIME); String[] data = new String[]{ "11:11:11", "23:12:50", "00:00:01", "09:59:59", "12:13:14" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminmaxtime", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtime"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "00:00:01,23:12:50\n"; @@ -141,7 +141,7 @@ public void testMinMaxTime() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminmaxtime PURGE"); } } @@ -154,10 +154,10 @@ public void testMinMaxTimeWithNull() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.TIME); String[] data = new String[]{ "11:11:11", "\\N", "\\N", "09:59:59", "12:13:14" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminmaxtimewithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtimewithnull"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "09:59:59,12:13:14\n"; @@ -165,7 +165,7 @@ public void testMinMaxTimeWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminmaxtimewithnull PURGE"); } } @@ -179,10 +179,10 @@ public void testMinMaxTimestamp() throws Exception { schema.addColumn("value", TajoDataTypes.Type.TIMESTAMP); String[] data = new String[]{ "1999-01-01 11:11:11", "2015-01-01 23:12:50", "2016-12-24 00:00:01", "1977-05-04 09:59:59", "2002-11-21 12:13:14" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminmaxtimestamp", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtimestamp"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "1977-05-04 09:59:59,2016-12-24 00:00:01\n"; @@ -190,7 +190,7 @@ public void testMinMaxTimestamp() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminmaxtimestamp PURGE"); } } @@ -204,10 +204,10 @@ public void testMinMaxTimestampWithNull() throws Exception { schema.addColumn("value", TajoDataTypes.Type.TIMESTAMP); String[] data = new String[]{ "1999-01-01 11:11:11", "2015-01-01 23:12:50", "\\N", "\\N", "2002-11-21 12:13:14" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminmaxtimestampwithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtimestampwithnull"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "1999-01-01 11:11:11,2015-01-01 23:12:50\n"; @@ -215,7 +215,7 @@ public void testMinMaxTimestampWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminmaxtimestampwithnull PURGE"); } } @@ -236,10 +236,10 @@ public void testMinLongWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("value", TajoDataTypes.Type.INT8); String[] data = new String[]{ "1|111", "2|\\N", "3|333" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminlongwithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value from testbuiltin11"); + ResultSet res = executeString("select min(value) as min_value from testminlongwithnull"); String ascExpected = "min_value\n" + "-------------------------------\n" + "111\n"; @@ -247,7 +247,7 @@ public void testMinLongWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminlongwithnull PURGE"); } } @@ -269,10 +269,10 @@ public void testMaxStringWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("name", TajoDataTypes.Type.TEXT); String[] data = new String[]{ "1|\\N", "2|\\N", "3|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testmaxstringwithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select max(name) as max_name from testbuiltin11"); + ResultSet res = executeString("select max(name) as max_name from testmaxstringwithnull"); String ascExpected = "max_name\n" + "-------------------------------\n" + "null\n"; @@ -280,7 +280,7 @@ public void testMaxStringWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testmaxstringwithnull PURGE"); } } @@ -302,10 +302,10 @@ public void testMinStringWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("name", TajoDataTypes.Type.TEXT); String[] data = new String[]{ "1|def", "2|\\N", "3|abc" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testminstringwithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(name) as min_name from testbuiltin11"); + ResultSet res = executeString("select min(name) as min_name from testminstringwithnull"); String ascExpected = "min_name\n" + "-------------------------------\n" + "abc\n"; @@ -313,7 +313,7 @@ public void testMinStringWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testminstringwithnull PURGE"); } } @@ -366,10 +366,10 @@ public void testAvgWithNull() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testavgwithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testbuiltin11"); + ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testavgwithnull"); String ascExpected = "avg_int,avg_long,avg_float,avg_double\n" + "-------------------------------\n" + "1.5,-222.0,2.0,1.0\n"; @@ -377,7 +377,7 @@ public void testAvgWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testavgwithnull PURGE"); } } @@ -395,10 +395,10 @@ public void testAvgWithAllNulls() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|\\N|\\N|\\N", "2|\\N|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testavgwithallnulls", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testbuiltin11"); + ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testavgwithallnulls"); String ascExpected = "avg_int,avg_long,avg_float,avg_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -406,7 +406,7 @@ public void testAvgWithAllNulls() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testavgwithallnulls PURGE"); } } @@ -424,10 +424,10 @@ public void testSumWithNull() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testsumwithnull", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testbuiltin11"); + ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testsumwithnull"); String ascExpected = "sum_int,sum_long,sum_float,sum_double\n" + "-------------------------------\n" + "3,-444,4.0,2.0\n"; @@ -435,7 +435,7 @@ public void testSumWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testsumwithnull PURGE"); } } @@ -453,10 +453,10 @@ public void testSumWithAllNulls() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|\\N|\\N|\\N", "2|\\N|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testsumwithallnulls", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testbuiltin11"); + ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testsumwithallnulls"); String ascExpected = "sum_int,sum_long,sum_float,sum_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -464,7 +464,7 @@ public void testSumWithAllNulls() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testsumwithallnulls PURGE"); } } @@ -485,10 +485,10 @@ public void testStdDevSamp() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("teststddevsamp", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from testbuiltin11"); + ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from teststddevsamp"); String ascExpected = "sdsamp_int,sdsamp_long,sdsamp_float,sdsamp_double\n" + "-------------------------------\n" + "0.7071067811865476,156.97770542341354,1.1313707824635184,72.8319984622144\n"; @@ -496,7 +496,7 @@ public void testStdDevSamp() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE teststddevsamp PURGE"); } } @@ -517,10 +517,10 @@ public void testStdDevSampWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("teststddevsampwithfewnumbers", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from testbuiltin11"); + ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from teststddevsampwithfewnumbers"); String ascExpected = "sdsamp_int,sdsamp_long,sdsamp_float,sdsamp_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -528,7 +528,7 @@ public void testStdDevSampWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE teststddevsampwithfewnumbers PURGE"); } } @@ -549,10 +549,10 @@ public void testStdDevPop() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("teststddevpop", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from testbuiltin11"); + ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from teststddevpop"); String ascExpected = "sdpop_int,sdpop_long,sdpop_float,sdpop_double\n" + "-------------------------------\n" + "0.5,111.0,0.7999999523162842,51.5\n"; @@ -560,7 +560,7 @@ public void testStdDevPop() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE teststddevpop PURGE"); } } @@ -581,10 +581,10 @@ public void testStdDevPopWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("teststddevpopwithfewnumbers", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from testbuiltin11"); + ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from teststddevpopwithfewnumbers"); String ascExpected = "sdpop_int,sdpop_long,sdpop_float,sdpop_double\n" + "-------------------------------\n" + "0.0,null,null,0.0\n"; @@ -592,7 +592,7 @@ public void testStdDevPopWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE teststddevpopwithfewnumbers PURGE"); } } @@ -613,10 +613,10 @@ public void testVarSamp() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testvarsamp", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_samp(value_int) as vs_int, var_samp(value_long) as vs_long, var_samp(value_float) as vs_float, var_samp(value_double) as vs_double from testbuiltin11"); + ResultSet res = executeString("select var_samp(value_int) as vs_int, var_samp(value_long) as vs_long, var_samp(value_float) as vs_float, var_samp(value_double) as vs_double from testvarsamp"); String ascExpected = "vs_int,vs_long,vs_float,vs_double\n" + "-------------------------------\n" + "0.5,24642.0,1.279999847412114,5304.5\n"; @@ -624,7 +624,7 @@ public void testVarSamp() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testvarsamp PURGE"); } } @@ -644,10 +644,10 @@ public void testVarSampWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testvarsampwithfewnumbers", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_samp(value_int) as vsamp_int, var_samp(value_long) as vsamp_long, var_samp(value_float) as vsamp_float, var_samp(value_double) as vsamp_double from testbuiltin11"); + ResultSet res = executeString("select var_samp(value_int) as vsamp_int, var_samp(value_long) as vsamp_long, var_samp(value_float) as vsamp_float, var_samp(value_double) as vsamp_double from testvarsampwithfewnumbers"); String ascExpected = "vsamp_int,vsamp_long,vsamp_float,vsamp_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -655,7 +655,7 @@ public void testVarSampWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testvarsampwithfewnumbers PURGE"); } } @@ -675,10 +675,10 @@ public void testVarPop() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testvarpop", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testbuiltin11"); + ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testvarpop"); String ascExpected = "vpop_int,vpop_long,vpop_float,vpop_double\n" + "-------------------------------\n" + "0.25,12321.0,0.639999923706057,2652.25\n"; @@ -686,7 +686,7 @@ public void testVarPop() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testvarpop PURGE"); } } @@ -706,10 +706,10 @@ public void testVarPopWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testvarpopwithfewnumbers", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testbuiltin11"); + ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testvarpopwithfewnumbers"); String ascExpected = "vpop_int,vpop_long,vpop_float,vpop_double\n" + "-------------------------------\n" + "0.0,null,null,0.0\n"; @@ -717,7 +717,7 @@ public void testVarPopWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testvarpopwithfewnumbers PURGE"); } } @@ -760,16 +760,16 @@ public void testRankWithTwoTables() throws Exception { Schema schema = new Schema(); schema.addColumn("id", TajoDataTypes.Type.INT4); String[] data = new String[] {"1", "3", "2", "4"}; - TajoTestingCluster.createTable("rank_table1", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testrankwithtwotables1", schema, tableOptions, data, 1); schema = new Schema(); schema.addColumn("refid", TajoDataTypes.Type.INT4); schema.addColumn("value", TajoDataTypes.Type.TEXT); data = new String[] {"1|efgh", "2|abcd", "4|erjk", "8|dfef"}; - TajoTestingCluster.createTable("rank_table2", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testrankwithtwotables2", schema, tableOptions, data, 1); ResultSet res = null; try { - res = executeString("select rank() over (order by id) from rank_table1 a, rank_table2 b " + res = executeString("select rank() over (order by id) from testrankwithtwotables1 a, testrankwithtwotables2 b " + " where a.id = b.refid"); String expectedString = "?windowfunction\n" + "-------------------------------\n" + @@ -784,8 +784,8 @@ public void testRankWithTwoTables() throws Exception { res.close(); } catch(Throwable ignored) {} } - executeString("DROP TABLE rank_table1 PURGE"); - executeString("DROP TABLE rank_table2 PURGE"); + executeString("DROP TABLE testrankwithtwotables1 PURGE"); + executeString("DROP TABLE testrankwithtwotables2 PURGE"); } } @@ -807,10 +807,10 @@ public void testCorr() throws Exception { "3|2|-333|2.8|\\N", "4|3|-555|2.8|43.2", "5|4|-111|1.1|10.2",}; - TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testcorr", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select corr(value_int, value_long) as corr1, corr(value_long, value_float) as corr2, corr(value_float, value_double) as corr3, corr(value_double, value_int) as corr4 from testbuiltin11"); + ResultSet res = executeString("select corr(value_int, value_long) as corr1, corr(value_long, value_float) as corr2, corr(value_float, value_double) as corr3, corr(value_double, value_int) as corr4 from testcorr"); String ascExpected = "corr1,corr2,corr3,corr4\n" + "-------------------------------\n" + "0.5,-0.9037045658322675,0.7350290063698216,-0.8761489936497805\n"; @@ -818,7 +818,7 @@ public void testCorr() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testbuiltin11 PURGE"); + executeString("DROP TABLE testcorr PURGE"); } } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java index bec8cd37e9..e07d5c5461 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java @@ -43,17 +43,17 @@ public void testCaseWhens1() throws IOException { schema.addColumn("col8", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); schema.addColumn("col9", TajoDataTypes.Type.INT4); - testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", - "select case when col1 between 1 and 3 then 10 else 100 end from table1;", + testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", + "select case when col1 between 1 and 3 then 10 else 100 end from testcasewhens1;", new String [] {"10"}); - testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", - "select case when col1 > 1 then 10 when col1 > 2 then 20 else 100 end from table1;", + testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", + "select case when col1 > 1 then 10 when col1 > 2 then 20 else 100 end from testcasewhens1;", new String [] {"100"}); - testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", - "select case col1 when 1 then 10 when 2 then 20 else 100 end from table1;", + testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", + "select case col1 when 1 then 10 when 2 then 20 else 100 end from testcasewhens1;", new String [] {"10"}); - testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", - "select case col9 when 1 then 10 when 2 then 20 else 100 end is null from table1;", + testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", + "select case col9 when 1 then 10 when 2 then 20 else 100 end is null from testcasewhens1;", new String [] {"f"}); } @@ -63,11 +63,11 @@ public void testCaseWhensWithNullReturn() throws IOException { schema.addColumn("col1", TajoDataTypes.Type.TEXT); schema.addColumn("col2", TajoDataTypes.Type.TEXT); - testEval(schema, "table1", "str1,str2", - "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM table1", + testEval(schema, "testcasewhenswithnullreturn", "str1,str2", + "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM testcasewhenswithnullreturn", new String[]{"str2"}); - testEval(schema, "table1", ",str2", - "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM table1", + testEval(schema, "testcasewhenswithnullreturn", ",str2", + "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM testcasewhenswithnullreturn", new String[]{""}); } @@ -78,34 +78,34 @@ public void testCaseWhensWithCommonExpression() throws IOException { schema.addColumn("col2", TajoDataTypes.Type.INT4); schema.addColumn("col3", TajoDataTypes.Type.INT4); - testEval(schema, "table1", "1,2,3", - "SELECT CASE WHEN col1 = 1 THEN 1 WHEN col1 = 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE WHEN col1 = 1 THEN 1 WHEN col1 = 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", new String [] {"1"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE WHEN col2 = 1 THEN 1 WHEN col2 = 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE WHEN col2 = 1 THEN 1 WHEN col2 = 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", new String [] {"2"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE WHEN col3 = 1 THEN 1 WHEN col3 = 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE WHEN col3 = 1 THEN 1 WHEN col3 = 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", new String [] {"3"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", new String [] {"1"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", new String [] {"2"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", new String [] {"3"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM testcasewhenswithcommonexpression", new String [] {"aaa"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM testcasewhenswithcommonexpression", new String [] {"bbb"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM table1", + testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM testcasewhenswithcommonexpression", new String [] {"ccc"}); } @@ -116,24 +116,24 @@ public void testCaseWhensWithCommonExpressionAndNull() throws IOException { schema.addColumn("col2", TajoDataTypes.Type.INT4); schema.addColumn("col3", TajoDataTypes.Type.INT4); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpressionandnull", new String [] {""}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpressionandnull", new String [] {"2"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM table1", + testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpressionandnull", new String [] {"3"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE NULL END FROM table1", + testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE NULL END FROM testcasewhenswithcommonexpressionandnull", new String [] {"1"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM table1", + testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM testcasewhenswithcommonexpressionandnull", new String [] {"2"}); - testEval(schema, "table1", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM table1", + testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM testcasewhenswithcommonexpressionandnull", new String [] {""}); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java index 25a10fd77c..17bc5bce22 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java @@ -35,7 +35,6 @@ import java.util.TimeZone; import static org.apache.tajo.common.TajoDataTypes.Type.*; -import static org.junit.Assert.assertEquals; public class TestDateTimeFunctions extends ExprTestBase { @Test @@ -122,40 +121,40 @@ public void testExtract() throws IOException { Schema schema2 = new Schema(); schema2.addColumn("col1", TIMESTAMP); - testEval(schema2, "table1", + testEval(schema2, "testextract", "1970-01-17 10:09:37", - "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "table1", + testEval(schema2, "testextract", "1970-01-17 10:09:37" + getUserTimeZoneDisplay(GMT), - "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "table1", + testEval(schema2, "testextract", "1970-01-17 10:09:37" + getUserTimeZoneDisplay(PST), - "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", new String[]{"1970.0", "1.0", "17.0"}); // Currently TIME type can be loaded with INT8 type. Schema schema3 = new Schema(); schema3.addColumn("col1", TIME); - testEval(schema3, "table1", + testEval(schema3, "testextract", "10:09:37.5", - "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;", + "select extract(hour from col1), extract(minute from col1), extract(second from col1) from testextract;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "table1", + testEval(schema3, "testextract", "10:09:37.5" + getUserTimeZoneDisplay(GMT), - "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;", + "select extract(hour from col1), extract(minute from col1), extract(second from col1) from testextract;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "table1", + testEval(schema3, "testextract", "10:09:37.5" + getUserTimeZoneDisplay(PST), - "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;", + "select extract(hour from col1), extract(minute from col1), extract(second from col1) from testextract;", new String[]{"18.0", "9.0", "37.5"}); Schema schema4 = new Schema(); schema4.addColumn("col1", DATE); - testEval(schema4, "table1", + testEval(schema4, "testextract", "1970-01-17", - "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", new String[]{"1970.0", "1.0", "17.0"}); testSimpleEval("select extract(century from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"20.0"}); @@ -239,36 +238,36 @@ public void testDatePart() throws IOException { Schema schema2 = new Schema(); schema2.addColumn("col1", TIMESTAMP); - testEval(schema2, "table1", + testEval(schema2, "testdatepart", "1970-01-17 22:09:37", - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "table1", + testEval(schema2, "testdatepart", "1970-01-17 22:09:37" + getUserTimeZoneDisplay(GMT), - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "table1", + testEval(schema2, "testdatepart", "1970-01-17 22:09:37" + getUserTimeZoneDisplay(PST), - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", new String[]{"1970.0", "1.0", "18.0"}); Schema schema3 = new Schema(); schema3.addColumn("col1", TIME); - testEval(schema3, "table1", "10:09:37.5", - "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;", + testEval(schema3, "testdatepart", "10:09:37.5", + "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from testdatepart;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "table1", "10:09:37.5" + getUserTimeZoneDisplay(GMT), - "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;", + testEval(schema3, "testdatepart", "10:09:37.5" + getUserTimeZoneDisplay(GMT), + "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from testdatepart;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "table1", "10:09:37.5" + getUserTimeZoneDisplay(PST), - "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;", + testEval(schema3, "testdatepart", "10:09:37.5" + getUserTimeZoneDisplay(PST), + "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from testdatepart;", new String[]{"18.0", "9.0", "37.5"}); Schema schema4 = new Schema(); schema4.addColumn("col1", DATE); - testEval(schema4, "table1", + testEval(schema4, "testdatepart", "1970-01-17", - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", new String[]{"1970.0", "1.0", "17.0"}); testSimpleEval("select date_part('century', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"20.0"}); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java index 78509f77e1..e0ea6b51a4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java @@ -42,7 +42,7 @@ public void testRound() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from table1", + testEval(schema, "testround", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from testround", new String[]{"2"}); Schema schema2 = new Schema(); @@ -51,8 +51,8 @@ public void testRound() throws IOException { schema2.addColumn("col3", FLOAT4); schema2.addColumn("col4", FLOAT8); - testEval(schema2, "table1", "9,9,9.5,9.5", - "select round(col1), round (col2), round(col3), round(col4) from table1", + testEval(schema2, "testround", "9,9,9.5,9.5", + "select round(col1), round (col2), round(col3), round(col4) from testround", new String [] {"9", "9", "10", "10"}); } @@ -70,7 +70,7 @@ public void testFloor() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from table1", + testEval(schema, "testfloor", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from testfloor", new String[]{"1"}); } @@ -89,7 +89,7 @@ public void testCeil() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from table1", + testEval(schema, "testceil", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from testceil", new String[]{"2"}); } @@ -108,7 +108,7 @@ public void testCeiling() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceiling(col1 + col2 + col3) from table1", + testEval(schema, "testceiling", "1.0, 0.2, 0.1", "select ceiling(col1 + col2 + col3) from testceiling", new String[]{"2"}); } @@ -124,7 +124,7 @@ public void testSin() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select sin(col1 + col2 + col3) from table1", + testEval(schema, "testsin", "1.0, 0.2, 0.1", "select sin(col1 + col2 + col3) from testsin", new String[]{"0.963558185417193"}); } @@ -141,7 +141,7 @@ public void testCos() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select cos(col1 + col2 + col3) from table1", + testEval(schema, "testcos", "1.0, 0.2, 0.1", "select cos(col1 + col2 + col3) from testcos", new String[]{"0.26749882862458735"}); } @@ -157,7 +157,7 @@ public void testTan() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select tan(col1 - col2 - col3) from table1", + testEval(schema, "testtan", "1.0, 0.2, 0.1", "select tan(col1 - col2 - col3) from testtan", new String[]{"0.8422883804630795"}); } @@ -173,7 +173,7 @@ public void testAsin() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select asin(col1 - col2 - col3) from table1", + testEval(schema, "testasin", "1.0, 0.2, 0.1", "select asin(col1 - col2 - col3) from testasin", new String[]{"0.7753974966107532"}); } @@ -189,7 +189,7 @@ public void testAcos() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select acos(col1 - col2 - col3) from table1", + testEval(schema, "testAcos", "1.0, 0.2, 0.1", "select acos(col1 - col2 - col3) from testAcos", new String[]{"0.7953988301841435"}); } @@ -205,7 +205,7 @@ public void testAtan() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan(col1 + col2 + col3) from table1", + testEval(schema, "testAtan", "1.0, 0.2, 0.1", "select atan(col1 + col2 + col3) from testAtan", new String[]{"0.9151007005533605"}); } @@ -222,7 +222,7 @@ public void testAtan2() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan2(col1 + col2, col3) from table1", + testEval(schema, "testAtan2", "1.0, 0.2, 0.1", "select atan2(col1 + col2, col3) from testAtan2", new String[]{"1.4876550949064553"}); } @@ -238,7 +238,7 @@ public void testMod() throws IOException { schema.addColumn("col2", INT8); schema.addColumn("col3", INT8); - testEval(schema, "table1", "9,2,3", "select mod(col1 + col2, col3) from table1", + testEval(schema, "testMod", "9,2,3", "select mod(col1 + col2, col3) from testMod", new String[]{"2"}); } @@ -254,7 +254,7 @@ public void testDiv() throws IOException { schema.addColumn("col2", INT8); schema.addColumn("col3", INT8); - testEval(schema, "table1", "9,2,3", "select div(col1 + col2, col3) from table1", + testEval(schema, "testDiv", "9,2,3", "select div(col1 + col2, col3) from testDiv", new String[]{"3"}); } @@ -279,7 +279,7 @@ public void testSign() throws IOException { schema2.addColumn("col2", FLOAT8); schema2.addColumn("col3", FLOAT8); - testEval(schema2, "table1", "1.0, 0.2, 0.1", "select sign(col1 + col2 + col3) from table1", + testEval(schema2, "testSign", "1.0, 0.2, 0.1", "select sign(col1 + col2 + col3) from testSign", new String[]{"1.0"}); } @@ -295,7 +295,7 @@ public void testSqrt() throws IOException { schema.addColumn("col2", FLOAT4); schema.addColumn("col3", FLOAT4); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from table1", + testEval(schema, "testSqrt", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from testSqrt", new String[]{"1.1401754564651765"}); @@ -304,7 +304,7 @@ public void testSqrt() throws IOException { schema2.addColumn("col2", FLOAT8); schema2.addColumn("col3", FLOAT8); - testEval(schema2, "table1", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from table1", + testEval(schema2, "testSqrt", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from testSqrt", new String[]{"1.140175425099138"}); } @@ -318,13 +318,13 @@ public void testExp() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", FLOAT4); - testEval(schema, "table1", "1.123", "select exp(col1) from table1", + testEval(schema, "testExp", "1.123", "select exp(col1) from testExp", new String[]{String.valueOf(Math.exp(1.123f))}); Schema schema2 = new Schema(); schema2.addColumn("col1", FLOAT8); - testEval(schema2, "table1", "1.123", "select exp(col1) from table1", + testEval(schema2, "testExp", "1.123", "select exp(col1) from testExp", new String[]{String.valueOf(Math.exp(1.123d))}); } @@ -341,12 +341,12 @@ public void testAbs() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", FLOAT4); schema.addColumn("col2", FLOAT4); - testEval(schema, "table1", "0.39,-0.39", "select abs(col1), abs(col2) from table1", new String[]{"0.39", "0.39"}); + testEval(schema, "testAbs", "0.39,-0.39", "select abs(col1), abs(col2) from testAbs", new String[]{"0.39", "0.39"}); Schema schema2 = new Schema(); schema2.addColumn("col1", FLOAT8); schema2.addColumn("col2", FLOAT8); - testEval(schema2, "table1", "0.033312347,-0.033312347", "select abs(col1), abs(col2) from table1", + testEval(schema2, "testAbs", "0.033312347,-0.033312347", "select abs(col1), abs(col2) from testAbs", new String[]{"0.033312347", "0.033312347"}); } @@ -361,14 +361,14 @@ public void testCbrt() throws IOException { schema.addColumn("col1", FLOAT4); schema.addColumn("col2", FLOAT4); schema.addColumn("col3", FLOAT4); - testEval(schema, "table1", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from table1", + testEval(schema, "testCbrt", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from testCbrt", new String[]{"1.0913929030771317"}); Schema schema2 = new Schema(); schema2.addColumn("col1", FLOAT8); schema2.addColumn("col2", FLOAT8); schema2.addColumn("col3", FLOAT8); - testEval(schema2, "table1", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from table1", + testEval(schema2, "testCbrt", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from testCbrt", new String[]{"1.091392883061106"}); } @@ -384,7 +384,7 @@ public void testDegrees() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "0.8,2.7,-0.8", "select degrees(col1), degrees(col2), degrees(col3) from table1", + testEval(schema, "testDegrees", "0.8,2.7,-0.8", "select degrees(col1), degrees(col2), degrees(col3) from testDegrees", new String[]{ String.valueOf(Math.toDegrees((float)0.8)), String.valueOf(Math.toDegrees(2.7)), @@ -406,7 +406,7 @@ public void testPow() throws IOException { schema.addColumn("col3", INT4); schema.addColumn("col4", INT8); - testEval(schema, "table1", "0.4,2.7,3,2", "select pow(col1, col2), pow(col3, col4) from table1", + testEval(schema, "testPow", "0.4,2.7,3,2", "select pow(col1, col2), pow(col3, col4) from testPow", new String[]{ String.valueOf(Math.pow((float) 0.4, 2.7)), String.valueOf(Math.pow(3, 2)) @@ -425,7 +425,7 @@ public void testRadians() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "table1", "0.8,2.7,-0.8", "select radians(col1), radians(col2), radians(col3) from table1", + testEval(schema, "testRadians", "0.8,2.7,-0.8", "select radians(col1), radians(col2), radians(col3) from testRadians", new String[]{ String.valueOf(Math.toRadians((float)0.8)), String.valueOf(Math.toRadians(2.7)), @@ -466,6 +466,6 @@ public void testRoundWithSpecifiedPrecision() throws IOException { schema.addColumn("col1", FLOAT8); schema.addColumn("col2", INT4); - testEval(schema, "table1", ",", "select round(col1, col2) from table1", new String[]{""}); + testEval(schema, "testRoundWithSpecifiedPrecision", ",", "select round(col1, col2) from testRoundWithSpecifiedPrecision", new String[]{""}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java index 8aae26d9c6..ba51858fb8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java @@ -34,27 +34,27 @@ public void testLike() throws IOException { schema.addColumn("col1", TEXT); // test for null values - testEval(schema, "table1", ",", "select col1 like 'a%' from table1", new String[]{""}); + testEval(schema, "testlike", ",", "select col1 like 'a%' from testlike", new String[]{""}); testSimpleEval("select null like 'a%'", new String[]{""}); - testEval(schema, "table1", "abc", "select col1 like '%c' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 like 'a%' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 like '_bc' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 like 'ab_' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 like '_b_' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 like '%b%' from table1", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 like '%c' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 like 'a%' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 like '_bc' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 like 'ab_' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 like '_b_' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 like '%b%' from testlike", new String[]{"t"}); // test for escaping regular expressions - testEval(schema, "table1", "abc", "select col1 not like '.bc' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 not like '.*bc' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 not like '.bc' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 not like '*bc' from table1", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like '.bc' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like '.*bc' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like '.bc' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like '*bc' from testlike", new String[]{"t"}); // test for case sensitive - testEval(schema, "table1", "abc", "select col1 not like '%C' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 not like 'A%' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 not like '_BC' from table1", new String[]{"t"}); - testEval(schema, "table1", "abc", "select col1 not like '_C_' from table1", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like '%C' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like 'A%' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like '_BC' from testlike", new String[]{"t"}); + testEval(schema, "testlike", "abc", "select col1 not like '_C_' from testlike", new String[]{"t"}); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java index 7f402a13bc..2727f1faad 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java @@ -47,8 +47,8 @@ public void testConcatenateOnExpressions() throws IOException { testSimpleEval("select (1+3) || 2 as col1 ", new String[]{"42"}); - testEval(schema, "table1", "abc,2,3.14", "select col1 || col2 || col3 from table1", new String[]{"abc23.14"}); - testEval(schema, "table1", "abc,2,3.14", "select col1 || '---' || col3 from table1", new String[]{"abc---3.14"}); + testEval(schema, "testconcatenateonexpressions", "abc,2,3.14", "select col1 || col2 || col3 from testconcatenateonexpressions", new String[]{"abc23.14"}); + testEval(schema, "testconcatenateonexpressions", "abc,2,3.14", "select col1 || '---' || col3 from testconcatenateonexpressions", new String[]{"abc---3.14"}); } @Test @@ -72,11 +72,11 @@ public void testLTrim() throws IOException { testSimpleEval("select trim(leading from ' trim') ", new String[]{"trim"}); testSimpleEval("select trim(' trim') ", new String[]{"trim"}); - testEval(schema, "table1", " trim,abc", "select ltrim(col1) from table1", new String[]{"trim"}); - testEval(schema, "table1", "xxtrim,abc", "select ltrim(col1, 'xx') from table1", new String[]{"trim"}); - testEval(schema, "table1", "xxtrim,abc", "select trim(leading 'xx' from col1) from table1", new String[]{"trim"}); + testEval(schema, "testLTrim", " trim,abc", "select ltrim(col1) from testltrim", new String[]{"trim"}); + testEval(schema, "testLTrim", "xxtrim,abc", "select ltrim(col1, 'xx') from testltrim", new String[]{"trim"}); + testEval(schema, "testLTrim", "xxtrim,abc", "select trim(leading 'xx' from col1) from testltrim", new String[]{"trim"}); - testEval(schema, "table1", " trim, abc", "select ltrim(col1) || ltrim(col2) from table1", + testEval(schema, "testLTrim", " trim, abc", "select ltrim(col1) || ltrim(col2) from testltrim", new String[]{"trimabc"}); } @@ -93,11 +93,11 @@ public void testRTrim() throws IOException { testSimpleEval("select trim(trailing from 'trim ') ", new String[]{"trim"}); testSimpleEval("select trim('trim ') ", new String[]{"trim"}); - testEval(schema, "table1", "trim ,abc", "select rtrim(col1) from table1", new String[]{"trim"}); - testEval(schema, "table1", "trimxx,abc", "select rtrim(col1, 'xx') from table1", new String[]{"trim"}); - testEval(schema, "table1", "trimxx,abc", "select trim(trailing 'xx' from col1) from table1", new String[]{"trim"}); + testEval(schema, "testRTrim", "trim ,abc", "select rtrim(col1) from testRTrim", new String[]{"trim"}); + testEval(schema, "testRTrim", "trimxx,abc", "select rtrim(col1, 'xx') from testRTrim", new String[]{"trim"}); + testEval(schema, "testRTrim", "trimxx,abc", "select trim(trailing 'xx' from col1) from testRTrim", new String[]{"trim"}); - testEval(schema, "table1", "trim ,abc ", "select rtrim(col1) || rtrim(col2) from table1", + testEval(schema, "testRTrim", "trim ,abc ", "select rtrim(col1) || rtrim(col2) from testRTrim", new String[]{"trimabc"}); } @@ -114,11 +114,11 @@ public void testTrim() throws IOException { testSimpleEval("select trim(both from ' trim ') ", new String[]{"trim"}); testSimpleEval("select trim(' trim ') ", new String[]{"trim"}); - testEval(schema, "table1", " trim ,abc", "select trim(col1) from table1", new String[]{"trim"}); - testEval(schema, "table1", "xxtrimxx,abc", "select trim(col1, 'xx') from table1", new String[]{"trim"}); - testEval(schema, "table1", "xxtrimxx,abc", "select trim(both 'xx' from col1) from table1", new String[]{"trim"}); + testEval(schema, "testtrim", " trim ,abc", "select trim(col1) from testtrim", new String[]{"trim"}); + testEval(schema, "testtrim", "xxtrimxx,abc", "select trim(col1, 'xx') from testtrim", new String[]{"trim"}); + testEval(schema, "testtrim", "xxtrimxx,abc", "select trim(both 'xx' from col1) from testtrim", new String[]{"trim"}); - testEval(schema, "table1", " trim ,xxabcxx", "select trim(col1) || trim(col2,'xx') from table1", + testEval(schema, "testtrim", " trim ,xxabcxx", "select trim(col1) || trim(col2,'xx') from testtrim", new String[]{"trimabc"}); } @@ -137,13 +137,13 @@ public void testRegexReplace() throws IOException { schema.addColumn("col3", TEXT); // find matches and replace from column values - testEval(schema, "table1", "------,(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from table1", + testEval(schema, "testregexreplace", "------,(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from testregexreplace", new String[]{"ab--ab"}); // null test from a table - testEval(schema, "table1", ",(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from table1", + testEval(schema, "testregexreplace", ",(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from testregexreplace", new String[]{""}); - testEval(schema, "table1", "------,(^--|--$),", "select regexp_replace(col1, col2, col3) as str from table1", + testEval(schema, "testregexreplace", "------,(^--|--$),", "select regexp_replace(col1, col2, col3) as str from testregexreplace", new String[]{""}); } @@ -170,10 +170,10 @@ public void testLeft() throws IOException { schema.addColumn("col3", TEXT); // for null tests - testEval(schema, "table1", ",1,ghi", "select left(col1,1) is null from table1", new String[]{"t"}); - testEval(schema, "table1", "abc,,ghi", "select left(col1,col2) is null from table1", new String[]{"t"}); + testEval(schema, "testleft", ",1,ghi", "select left(col1,1) is null from testleft", new String[]{"t"}); + testEval(schema, "testleft", "abc,,ghi", "select left(col1,col2) is null from testleft", new String[]{"t"}); - testEval(schema, "table1", "abc,1,ghi", "select left(col1,1) || left(col3,3) from table1", new String[]{"aghi"}); + testEval(schema, "testleft", "abc,1,ghi", "select left(col1,1) || left(col3,3) from testleft", new String[]{"aghi"}); } @Test @@ -199,10 +199,10 @@ public void testRight() throws IOException { schema.addColumn("col3", TEXT); // for null tests - testEval(schema, "table1", ",1,ghi", "select right(col1,1) is null from table1", new String[]{"t"}); - testEval(schema, "table1", "abc,,ghi", "select right(col1,col2) is null from table1", new String[]{"t"}); + testEval(schema, "testright", ",1,ghi", "select right(col1,1) is null from testright", new String[]{"t"}); + testEval(schema, "testright", "abc,,ghi", "select right(col1,col2) is null from testright", new String[]{"t"}); - testEval(schema, "table1", "abc,1,ghi", "select right(col1,1) || right(col3,3) from table1", new String[]{"cghi"}); + testEval(schema, "testright", "abc,1,ghi", "select right(col1,1) || right(col3,3) from testright", new String[]{"cghi"}); } @Test @@ -214,7 +214,7 @@ public void testReverse() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "abc,efg,3.14", "select reverse(col1) || reverse(col2) from table1", + testEval(schema, "testReverse", "abc,efg,3.14", "select reverse(col1) || reverse(col2) from testReverse", new String[]{"cbagfe"}); } @@ -228,7 +228,7 @@ public void testRepeat() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "abc,efg,3.14", "select repeat(col1,2) from table1", new String[]{"abcabc"}); + testEval(schema, "testrepeat", "abc,efg,3.14", "select repeat(col1,2) from testrepeat", new String[]{"abcabc"}); } @@ -240,9 +240,9 @@ public void testUpper() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "abc,efg,3.14", "select upper(col1), upper(col2) from table1", + testEval(schema, "testupper", "abc,efg,3.14", "select upper(col1), upper(col2) from testupper", new String[]{"ABC", "EFG"}); - testEval(schema, "table1", "abc,efg,3.14", "select upper(col1) || upper(col2) from table1", new String[]{"ABCEFG"}); + testEval(schema, "testupper", "abc,efg,3.14", "select upper(col1) || upper(col2) from testupper", new String[]{"ABCEFG"}); } @Test @@ -253,9 +253,9 @@ public void testLower() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "ABC,DEF,3.14", "select lower(col1), lower(col2) from table1", + testEval(schema, "testlower", "ABC,DEF,3.14", "select lower(col1), lower(col2) from testlower", new String[]{"abc", "def"}); - testEval(schema, "table1", "ABC,DEF,3.14", "select lower(col1) || lower(col2) from table1", new String[]{"abcdef"}); + testEval(schema, "testlower", "ABC,DEF,3.14", "select lower(col1) || lower(col2) from testlower", new String[]{"abcdef"}); } @Test @@ -266,7 +266,7 @@ public void testCharLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "ABC,DEF,3.14", "select character_length(lower(col1) || lower(col2)) from table1", + testEval(schema, "testcharlength", "ABC,DEF,3.14", "select character_length(lower(col1) || lower(col2)) from testcharlength", new String[]{"6"}); } @@ -278,7 +278,7 @@ public void testLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "ABC,DEF,3.14", "select length(lower(col1) || lower(col2)) from table1", + testEval(schema, "testlength", "ABC,DEF,3.14", "select length(lower(col1) || lower(col2)) from testlength", new String[]{"6"}); } @@ -291,7 +291,7 @@ public void testMd5() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "abc,efg,3.14", "select md5(col1) from table1", + testEval(schema, "testmd5", "abc,efg,3.14", "select md5(col1) from testmd5", new String[]{"900150983cd24fb0d6963f7d28e17f72"}); } @@ -320,7 +320,7 @@ public void testHex() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", ",abcdef,3.14", "select to_hex(10) from table1", + testEval(schema, "testhex", ",abcdef,3.14", "select to_hex(10) from testhex", new String[]{"a"}); } @@ -334,7 +334,7 @@ public void testBin() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", ",abcdef,3.14", "select to_bin(20) from table1", + testEval(schema, "testbin", ",abcdef,3.14", "select to_bin(20) from testbin", new String[]{"10100"}); } @@ -348,7 +348,7 @@ public void testOctetLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "ABC,DEF,3.14", "select octet_length(lower(col1) || lower(col2)) from table1", + testEval(schema, "testoctetlength", "ABC,DEF,3.14", "select octet_length(lower(col1) || lower(col2)) from testoctetlength", new String[]{"6"}); } @@ -364,10 +364,10 @@ public void testSplitPart() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "t1", ",.,1", "select split_part(col1, col2, col3::int) is null from t1", new String[]{"t"}); - testEval(schema, "t1", "1386577650.123,,1", "select split_part(col1, col2, col3::int) from t1", + testEval(schema, "testsplitpart", ",.,1", "select split_part(col1, col2, col3::int) is null from testsplitpart", new String[]{"t"}); + testEval(schema, "testsplitpart", "1386577650.123,,1", "select split_part(col1, col2, col3::int) from testsplitpart", new String[]{"1386577650.123"}); - testEval(schema, "t1", "1386577650.123,.,", "select split_part(col1, col2, col3::int) is null from t1", + testEval(schema, "testsplitpart", "1386577650.123,.,", "select split_part(col1, col2, col3::int) is null from testsplitpart", new String[]{"t"}); } @@ -393,7 +393,7 @@ public void testSubstr() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", ",abcdef,3.14", "select substr(lower(col2), 2, 3) from table1", + testEval(schema, "testsubstr", ",abcdef,3.14", "select substr(lower(col2), 2, 3) from testsubstr", new String[]{"bcd"}); } @@ -436,14 +436,14 @@ public void testLocate() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd') from table1", new String[]{"3"}); - testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd', 1) from table1", new String[]{"3"}); - testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd', 4) from table1", new String[]{"0"}); - testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'xy') from table1", new String[]{"0"}); + testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'cd') from testlocate", new String[]{"3"}); + testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'cd', 1) from testlocate", new String[]{"3"}); + testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'cd', 4) from testlocate", new String[]{"0"}); + testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'xy') from testlocate", new String[]{"0"}); // null string - testEval(schema, "table1", ",abcdef,3.14", "select locate(col1, 'cd') is null from table1", new String[]{"t"}); + testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col1, 'cd') is null from testlocate", new String[]{"t"}); // nul substring - testEval(schema, "table1", ",abcdef,3.14", "select locate('cd', col1) is null from table1", new String[]{"t"}); + testEval(schema, "testlocate", ",abcdef,3.14", "select locate('cd', col1) is null from testlocate", new String[]{"t"}); } @Test @@ -454,7 +454,7 @@ public void testBitLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "ABC,DEF,3.14", "select bit_length(lower(col1) || lower(col2)) from table1", + testEval(schema, "testbitlength", "ABC,DEF,3.14", "select bit_length(lower(col1) || lower(col2)) from testbitlength", new String[]{"48"}); } @@ -470,7 +470,7 @@ public void testStrpos() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strpos(lower(col1) || lower(col2), 'fh') from table1", + testEval(schema, "teststrpos", "ABCDEF,HIJKLMN,3.14", "select strpos(lower(col1) || lower(col2), 'fh') from teststrpos", new String[]{"6"}); } @@ -486,7 +486,7 @@ public void testStrposb() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strposb(lower(col1) || lower(col2), 'fh') from table1", + testEval(schema, "teststrposb", "ABCDEF,HIJKLMN,3.14", "select strposb(lower(col1) || lower(col2), 'fh') from teststrposb", new String[]{"6"}); } @@ -502,9 +502,9 @@ public void testAscii() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TEXT); - testEval(schema, "table1", "abc", "select ascii(col1) from table1", + testEval(schema, "testascii", "abc", "select ascii(col1) from testascii", new String[]{"97"}); - testEval(schema, "table1", "12", "select ascii(col1) from table1", + testEval(schema, "testascii", "12", "select ascii(col1) from testascii", new String[]{"49"}); } @@ -518,9 +518,9 @@ public void testChr() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", INT4); - testEval(schema, "table1", "65", "select chr(col1) from table1", new String[]{"A"}); - testEval(schema, "table1", "66", "select chr(col1) from table1", new String[]{"B"}); - testEval(schema, "table1", "52512", "select chr(col1) from table1", new String[]{"ì´ "}); + testEval(schema, "testchr", "65", "select chr(col1) from testchr", new String[]{"A"}); + testEval(schema, "testchr", "66", "select chr(col1) from testchr", new String[]{"B"}); + testEval(schema, "testchr", "52512", "select chr(col1) from testchr", new String[]{"ì´ "}); } @Test @@ -590,9 +590,9 @@ public void testFindInSet() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); - testEval(schema, "table1", "|crt,c,cr,c,def", "select find_in_set(col1, col2) is null from table1", + testEval(schema, "testfindinset", "|crt,c,cr,c,def", "select find_in_set(col1, col2) is null from testfindinset", new String[]{"t"}, '|', true); - testEval(schema, "table1", "cr|", "select find_in_set(col1, col2) is null from table1", + testEval(schema, "testfindinset", "cr|", "select find_in_set(col1, col2) is null from testfindinset", new String[]{"t"}, '|', true); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java index ace3d0d901..0e9eb074b0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; +import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -28,54 +29,53 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.*; +import org.apache.tajo.engine.planner.PhysicalPlanner; +import org.apache.tajo.engine.planner.PhysicalPlannerImpl; import org.apache.tajo.engine.planner.enforce.Enforcer; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.After; -import org.junit.Before; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; -import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class TestBNLJoinExec { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestBNLJoinExec"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private LogicalPlanner planner; - private Path testDir; + private static TajoConf conf; + private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestBNLJoinExec"; + private static TajoTestingCluster util; + private static CatalogService catalog; + private static LogicalPlanner planner; + private static Path testDir; private static int OUTER_TUPLE_NUM = 1000; private static int INNER_TUPLE_NUM = 1000; - private TableDesc employee; - private TableDesc people; + private static TableDesc employee; + private static TableDesc people; - @Before - public void setUp() throws Exception { - util = new TajoTestingCluster(); + @BeforeClass + public static void setUp() throws Exception { + util = TpchTestBase.getInstance().getTestingCluster(); catalog = util.startCatalogCluster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); + catalog.createDatabase("testbnljoinexec", DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); Schema schema = new Schema(); @@ -98,7 +98,7 @@ public void setUp() throws Exception { } appender.flush(); appender.close(); - employee = CatalogUtil.newTableDesc("default.employee", schema, employeeMeta, employeePath); + employee = CatalogUtil.newTableDesc("testbnljoinexec.employee", schema, employeeMeta, employeePath); catalog.createTable(employee); Schema peopleSchema = new Schema(); @@ -121,26 +121,27 @@ public void setUp() throws Exception { appender.flush(); appender.close(); - people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath); + people = CatalogUtil.newTableDesc("testbnljoinexec.people", peopleSchema, peopleMeta, peoplePath); catalog.createTable(people); - analyzer = new SQLAnalyzer(); planner = new LogicalPlanner(catalog, TablespaceManager.getInstance()); } - @After - public void tearDown() throws Exception { - util.shutdownCatalogCluster(); + @AfterClass + public static void tearDown() throws Exception { + catalog.dropDatabase("testbnljoinexec"); + testDir.getFileSystem(conf).delete(testDir, true); } // employee (managerId, empId, memId, deptName) // people (empId, fk_memId, name, age) String[] QUERIES = { - "select managerId, e.empId, deptName, e.memId from employee as e, people p", - "select managerId, e.empId, deptName, e.memId from employee as e " + - "inner join people as p on e.empId = p.empId and e.memId = p.fk_memId" }; + "select managerId, e.empId, deptName, e.memId from testbnljoinexec.employee as e, testbnljoinexec.people p", + "select managerId, e.empId, deptName, e.memId from testbnljoinexec.employee as e " + + "inner join testbnljoinexec.people as p on e.empId = p.empId and e.memId = p.fk_memId" }; @Test public final void testBNLCrossJoin() throws IOException, PlanningException { + SQLAnalyzer analyzer = new SQLAnalyzer(); Expr expr = analyzer.parse(QUERIES[0]); LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr).getRootBlock().getRoot(); @@ -148,10 +149,10 @@ public final void testBNLCrossJoin() throws IOException, PlanningException { Enforcer enforcer = new Enforcer(); enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN); - FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(), + FileFragment[] empFrags = FileTablespace.splitNG(conf, "testbnljoinexec.e", employee.getMeta(), new Path(employee.getUri()), Integer.MAX_VALUE); - FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(), + FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "testbnljoinexec.p", people.getMeta(), new Path(people.getUri()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(empFrags, peopleFrags); @@ -177,13 +178,14 @@ public final void testBNLCrossJoin() throws IOException, PlanningException { @Test public final void testBNLInnerJoin() throws IOException, PlanningException { + SQLAnalyzer analyzer = new SQLAnalyzer(); Expr context = analyzer.parse(QUERIES[1]); LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), context).getRootBlock().getRoot(); - FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(), + FileFragment[] empFrags = FileTablespace.splitNG(conf, "testbnljoinexec.e", employee.getMeta(), new Path(employee.getUri()), Integer.MAX_VALUE); - FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(), + FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "testbnljoinexec.p", people.getMeta(), new Path(people.getUri()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(empFrags, peopleFrags); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java index dc4dd042b8..3ad14c349a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; +import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -28,59 +29,61 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.*; +import org.apache.tajo.engine.planner.PhysicalPlanner; +import org.apache.tajo.engine.planner.PhysicalPlannerImpl; import org.apache.tajo.engine.planner.enforce.Enforcer; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.storage.*; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.storage.Appender; +import org.apache.tajo.storage.FileTablespace; +import org.apache.tajo.storage.TablespaceManager; +import org.apache.tajo.storage.VTuple; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.After; -import org.junit.Before; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; -import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestFullOuterHashJoinExec { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterHashJoinExec"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private LogicalPlanner planner; - private Path testDir; - private QueryContext defaultContext; - - private TableDesc dep3; - private TableDesc job3; - private TableDesc emp3; - private TableDesc phone3; - - private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); - private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); - private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); - private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); - - @Before - public void setUp() throws Exception { - util = new TajoTestingCluster(); - util.initTestDir(); - catalog = util.startCatalogCluster().getCatalog(); + private static TajoConf conf; + private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterHashJoinExec"; + private static TajoTestingCluster util; + private static CatalogService catalog; + private static SQLAnalyzer analyzer; + private static LogicalPlanner planner; + private static Path testDir; + private static QueryContext defaultContext; + + private static TableDesc dep3; + private static TableDesc job3; + private static TableDesc emp3; + private static TableDesc phone3; + + private static final String DATABASENAME = "testfullouterhashjoinexec"; + private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); + private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); + private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); + private static final String PHONE3_NAME = CatalogUtil.buildFQName(DATABASENAME, "phone3"); + + @BeforeClass + public static void setUp() throws Exception { + util = TpchTestBase.getInstance().getTestingCluster(); + catalog = util.getMaster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); + catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -239,20 +242,25 @@ public void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @After - public void tearDown() throws Exception { - util.shutdownCatalogCluster(); + @AfterClass + public static void tearDown() throws Exception { + catalog.dropDatabase(DATABASENAME); + testDir.getFileSystem(conf).delete(testDir, true); } String[] QUERIES = { // [0] no nulls - "select dep3.dep_id, dep_name, emp_id, salary from dep3 full outer join emp3 on dep3.dep_id = emp3.dep_id", + String.format("select dep3.dep_id, dep_name, emp_id, salary from %s.dep3 full outer join %s.emp3 on %s.dep3.dep_id = %s.emp3.dep_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [1] nulls on the right operand - "select job3.job_id, job_title, emp_id, salary from job3 full outer join emp3 on job3.job_id=emp3.job_id", + String.format("select job3.job_id, job_title, emp_id, salary from %s.job3 full outer join %s.emp3 on %s.job3.job_id=%s.emp3.job_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [2] nulls on the left side - "select job3.job_id, job_title, emp_id, salary from emp3 full outer join job3 on job3.job_id=emp3.job_id", + String.format("select job3.job_id, job_title, emp_id, salary from %s.emp3 full outer join %s.job3 on %s.job3.job_id=%s.emp3.job_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [3] one operand is empty - "select emp3.emp_id, first_name, phone_number from emp3 full outer join phone3 on emp3.emp_id = phone3.emp_id" + String.format("select emp3.emp_id, first_name, phone_number from %s.emp3 full outer join %s.phone3 on %s.emp3.emp_id = %s.phone3.emp_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java index 8fd61d0de0..061cd85292 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java @@ -20,8 +20,8 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; -import org.apache.tajo.TajoConstants; import org.apache.tajo.TajoTestingCluster; +import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -39,53 +39,53 @@ import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.storage.*; +import org.apache.tajo.storage.Appender; +import org.apache.tajo.storage.FileTablespace; +import org.apache.tajo.storage.TablespaceManager; +import org.apache.tajo.storage.VTuple; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.After; -import org.junit.Before; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; -import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestFullOuterMergeJoinExec { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterMergeJoinExec"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private LogicalPlanner planner; - private Path testDir; - private QueryContext defaultContext; - - private TableDesc dep3; - private TableDesc dep4; - private TableDesc job3; - private TableDesc emp3; - private TableDesc phone3; - - private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); - private final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4"); - private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); - private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); - private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); - - @Before - public void setUp() throws Exception { - util = new TajoTestingCluster(); - util.initTestDir(); - catalog = util.startCatalogCluster().getCatalog(); + private static TajoConf conf; + private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterMergeJoinExec"; + private static TajoTestingCluster util; + private static CatalogService catalog; + private static SQLAnalyzer analyzer; + private static LogicalPlanner planner; + private static Path testDir; + private static QueryContext defaultContext; + + private static TableDesc dep3; + private static TableDesc dep4; + private static TableDesc job3; + private static TableDesc emp3; + private static TableDesc phone3; + + private static final String DATABASENAME = "testfulloutermergejoinexec"; + private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); + private static final String DEP4_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep4"); + private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); + private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); + private static final String PHONE3_NAME = CatalogUtil.buildFQName(DATABASENAME, "phone3"); + + @BeforeClass + public static void setUp() throws Exception { + util = TpchTestBase.getInstance().getTestingCluster(); + catalog = util.getMaster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); - + catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -283,24 +283,32 @@ public void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @After - public void tearDown() throws Exception { - util.shutdownCatalogCluster(); + + @AfterClass + public static void tearDown() throws Exception { + catalog.dropDatabase(DATABASENAME); + testDir.getFileSystem(conf).delete(testDir, true); } String[] QUERIES = { // [0] no nulls - "select dep3.dep_id, dep_name, emp_id, salary from emp3 full outer join dep3 on dep3.dep_id = emp3.dep_id", + String.format("select %s.dep3.dep_id, dep_name, emp_id, salary from %s.emp3 full outer join %s.dep3 on %s.dep3.dep_id = %s.emp3.dep_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [1] nulls on the left operand - "select job3.job_id, job_title, emp_id, salary from emp3 full outer join job3 on job3.job_id=emp3.job_id", + String.format("select %s.job3.job_id, job_title, emp_id, salary from %s.emp3 full outer join %s.job3 on %s.job3.job_id=%s.emp3.job_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [2] nulls on the right side - "select job3.job_id, job_title, emp_id, salary from job3 full outer join emp3 on job3.job_id=emp3.job_id", + String.format("select %s.job3.job_id, job_title, emp_id, salary from %s.job3 full outer join %s.emp3 on %s.job3.job_id=%s.emp3.job_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [3] no nulls, right continues after left - "select dep4.dep_id, dep_name, emp_id, salary from emp3 full outer join dep4 on dep4.dep_id = emp3.dep_id", + String.format("select %s.dep4.dep_id, dep_name, emp_id, salary from %s.emp3 full outer join %s.dep4 on %s.dep4.dep_id = %s.emp3.dep_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [4] one operand is empty - "select emp3.emp_id, first_name, phone_number from emp3 full outer join phone3 on emp3.emp_id = phone3.emp_id", + String.format("select %s.emp3.emp_id, first_name, phone_number from %s.emp3 full outer join %s.phone3 on %s.emp3.emp_id = %s.phone3.emp_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [5] one operand is empty - "select emp3.emp_id, first_name, phone_number from phone3 full outer join emp3 on emp3.emp_id = phone3.emp_id", + String.format("select %s.emp3.emp_id, first_name, phone_number from %s.phone3 full outer join %s.emp3 on %s.emp3.emp_id = %s.phone3.emp_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java index c93a1b40b6..ca5dbbc3f9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; +import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -28,60 +29,60 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.*; +import org.apache.tajo.engine.planner.PhysicalPlanner; +import org.apache.tajo.engine.planner.PhysicalPlannerImpl; import org.apache.tajo.engine.planner.enforce.Enforcer; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.After; -import org.junit.Before; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; -import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class TestLeftOuterHashJoinExec { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuterHashJoinExec"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private LogicalPlanner planner; - private Path testDir; - private QueryContext defaultContext; - - private TableDesc dep3; - private TableDesc job3; - private TableDesc emp3; - private TableDesc phone3; - - private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); - private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); - private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); - private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); - - @Before - public void setUp() throws Exception { - util = new TajoTestingCluster(); - util.initTestDir(); - catalog = util.startCatalogCluster().getCatalog(); + + private static TajoConf conf; + private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuterHashJoinExec"; + private static TajoTestingCluster util; + private static CatalogService catalog; + private static SQLAnalyzer analyzer; + private static LogicalPlanner planner; + private static Path testDir; + private static QueryContext defaultContext; + + private static TableDesc dep3; + private static TableDesc job3; + private static TableDesc emp3; + private static TableDesc phone3; + + private static final String DATABASENAME = "testleftouterhashjoinexec"; + private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); + private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); + private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); + private static final String PHONE3_NAME = CatalogUtil.buildFQName(DATABASENAME, "phone3"); + + @BeforeClass + public static void setUp() throws Exception { + util = TpchTestBase.getInstance().getTestingCluster(); + catalog = util.getMaster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); + catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -243,22 +244,28 @@ public void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @After - public void tearDown() throws Exception { - util.shutdownCatalogCluster(); + @AfterClass + public static void tearDown() throws Exception { + catalog.dropDatabase(DATABASENAME); + testDir.getFileSystem(conf).delete(testDir, true); } String[] QUERIES = { // [0] no nulls - "select dep3.dep_id, dep_name, emp_id, salary from dep3 left outer join emp3 on dep3.dep_id = emp3.dep_id", + String.format("select dep3.dep_id, dep_name, emp_id, salary from %s.dep3 left outer join %s.emp3 on %s.dep3.dep_id = %s.emp3.dep_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [1] nulls on the right operand - "select job3.job_id, job_title, emp_id, salary from job3 left outer join emp3 on job3.job_id=emp3.job_id", + String.format("select job3.job_id, job_title, emp_id, salary from %s.job3 left outer join %s.emp3 on %s.job3.job_id=%s.emp3.job_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [2] nulls on the left side - "select job3.job_id, job_title, emp_id, salary from emp3 left outer join job3 on job3.job_id=emp3.job_id", + String.format("select job3.job_id, job_title, emp_id, salary from %s.emp3 left outer join %s.job3 on %s.job3.job_id=%s.emp3.job_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [3] one operand is empty - "select emp3.emp_id, first_name, phone_number from emp3 left outer join phone3 on emp3.emp_id = phone3.emp_id", + String.format("select emp3.emp_id, first_name, phone_number from %s.emp3 left outer join %s.phone3 on %s.emp3.emp_id = %s.phone3.emp_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), // [4] one operand is empty - "select phone_number, emp3.emp_id, first_name from phone3 left outer join emp3 on emp3.emp_id = phone3.emp_id" + String.format("select phone_number, emp3.emp_id, first_name from %s.phone3 left outer join %s.emp3 on %s.emp3.emp_id = %s.phone3.emp_id", + DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java index f581db86de..3a13342001 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; +import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -40,43 +41,42 @@ import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.After; -import org.junit.Before; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; -import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.junit.Assert.assertEquals; // this is not a physical operator in itself, but it uses the HashLeftOuterJoinExec with switched inputs order public class TestRightOuterHashJoinExec { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterHashJoinExec"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private LogicalPlanner planner; - private Path testDir; - private QueryContext defaultContext; - - private TableDesc dep3; - private TableDesc job3; - private TableDesc emp3; - - private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); - private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); - private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); - - @Before - public void setUp() throws Exception { - util = new TajoTestingCluster(); - util.initTestDir(); - catalog = util.startCatalogCluster().getCatalog(); + + private static TajoConf conf; + private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterHashJoinExec"; + private static TajoTestingCluster util; + private static CatalogService catalog; + private static SQLAnalyzer analyzer; + private static LogicalPlanner planner; + private static Path testDir; + private static QueryContext defaultContext; + + private static TableDesc dep3; + private static TableDesc job3; + private static TableDesc emp3; + + private static final String DATABASENAME = "testrightouterhashjoinexec"; + private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); + private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); + private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); + + @BeforeClass + public static void setUp() throws Exception { + util = TpchTestBase.getInstance().getTestingCluster(); + catalog = util.getMaster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); + catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -216,15 +216,22 @@ public void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @After - public void tearDown() throws Exception { - util.shutdownCatalogCluster(); + @AfterClass + public static void tearDown() throws Exception { + catalog.dropDatabase(DATABASENAME); + testDir.getFileSystem(conf).delete(testDir, true); } String[] QUERIES = { - "select dep3.dep_id, dep_name, emp_id, salary from emp3 right outer join dep3 on dep3.dep_id = emp3.dep_id", //0 no nulls - "select job3.job_id, job_title, emp_id, salary from emp3 right outer join job3 on job3.job_id=emp3.job_id", //1 nulls on the left operand - "select job3.job_id, job_title, emp_id, salary from job3 right outer join emp3 on job3.job_id=emp3.job_id" //2 nulls on the right side + //0 no nulls + String.format("select dep3.dep_id, dep_name, emp_id, salary from %s.emp3 right outer join %s.dep3 on dep3.dep_id = emp3.dep_id", + DATABASENAME, DATABASENAME), + //1 nulls on the left operand + String.format("select job3.job_id, job_title, emp_id, salary from %s.emp3 right outer join %s.job3 on job3.job_id=emp3.job_id", + DATABASENAME, DATABASENAME), + //2 nulls on the right side + String.format("select job3.job_id, job_title, emp_id, salary from %s.job3 right outer join %s.emp3 on job3.job_id=emp3.job_id", + DATABASENAME, DATABASENAME) }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java index d86b2294bb..23b95ff18d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java @@ -28,22 +28,26 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.*; +import org.apache.tajo.engine.planner.PhysicalPlanner; +import org.apache.tajo.engine.planner.PhysicalPlannerImpl; import org.apache.tajo.engine.planner.enforce.Enforcer; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.storage.*; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.storage.Appender; +import org.apache.tajo.storage.FileTablespace; +import org.apache.tajo.storage.TablespaceManager; +import org.apache.tajo.storage.VTuple; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.After; -import org.junit.Before; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; @@ -54,30 +58,30 @@ import static org.junit.Assert.*; public class TestRightOuterMergeJoinExec { - private TajoConf conf; - private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterMergeJoinExec"; - private TajoTestingCluster util; - private CatalogService catalog; - private SQLAnalyzer analyzer; - private LogicalPlanner planner; - private Path testDir; - private QueryContext defaultContext; - - - private TableDesc dep3; - private TableDesc dep4; - private TableDesc job3; - private TableDesc emp3; - private TableDesc phone3; - - private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); - private final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4"); - private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); - private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); - private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); - - @Before - public void setUp() throws Exception { + private static TajoConf conf; + private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterMergeJoinExec"; + private static TajoTestingCluster util; + private static CatalogService catalog; + private static SQLAnalyzer analyzer; + private static LogicalPlanner planner; + private static Path testDir; + private static QueryContext defaultContext; + + + private static TableDesc dep3; + private static TableDesc dep4; + private static TableDesc job3; + private static TableDesc emp3; + private static TableDesc phone3; + + private static final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); + private static final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4"); + private static final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); + private static final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); + private static final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); + + @BeforeClass + public static void setUp() throws Exception { util = new TajoTestingCluster(); util.initTestDir(); catalog = util.startCatalogCluster().getCatalog(); @@ -286,8 +290,8 @@ public void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @After - public void tearDown() throws Exception { + @AfterClass + public static void tearDown() throws Exception { util.shutdownCatalogCluster(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java index b19a488cb4..906d8a8b59 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java @@ -58,35 +58,35 @@ public final void testCreateTable1() throws Exception { @Test public final void testCreateTable2() throws Exception { - executeString("CREATE DATABASE D1;").close(); - executeString("CREATE DATABASE D2;").close(); - - executeString("CREATE TABLE D1.table1 (age int);").close(); - executeString("CREATE TABLE D1.table2 (age int);").close(); - executeString("CREATE TABLE d2.table3 (age int);").close(); - executeString("CREATE TABLE d2.table4 (age int);").close(); - - assertTableExists("d1.table1"); - assertTableExists("d1.table2"); - assertTableNotExists("d2.table1"); - assertTableNotExists("d2.table2"); - - assertTableExists("d2.table3"); - assertTableExists("d2.table4"); - assertTableNotExists("d1.table3"); - assertTableNotExists("d1.table4"); - - executeString("DROP TABLE D1.table1"); - executeString("DROP TABLE D1.table2"); - executeString("DROP TABLE D2.table3"); - executeString("DROP TABLE D2.table4"); - - assertDatabaseExists("d1"); - assertDatabaseExists("d2"); - executeString("DROP DATABASE D1").close(); - executeString("DROP DATABASE D2").close(); - assertDatabaseNotExists("d1"); - assertDatabaseNotExists("d2"); + executeString("CREATE DATABASE testcreatetable2;").close(); + executeString("CREATE DATABASE testcreatetable2_2;").close(); + + executeString("CREATE TABLE testcreatetable2.table1 (age int);").close(); + executeString("CREATE TABLE testcreatetable2.table2 (age int);").close(); + executeString("CREATE TABLE testcreatetable2_2.table3 (age int);").close(); + executeString("CREATE TABLE testcreatetable2_2.table4 (age int);").close(); + + assertTableExists("testcreatetable2.table1"); + assertTableExists("testcreatetable2.table2"); + assertTableNotExists("testcreatetable2_2.table1"); + assertTableNotExists("testcreatetable2_2.table2"); + + assertTableExists("testcreatetable2_2.table3"); + assertTableExists("testcreatetable2_2.table4"); + assertTableNotExists("testcreatetable2.table3"); + assertTableNotExists("testcreatetable2.table4"); + + executeString("DROP TABLE testcreatetable2.table1"); + executeString("DROP TABLE testcreatetable2.table2"); + executeString("DROP TABLE testcreatetable2_2.table3"); + executeString("DROP TABLE testcreatetable2_2.table4"); + + assertDatabaseExists("testcreatetable2"); + assertDatabaseExists("testcreatetable2_2"); + executeString("DROP DATABASE testcreatetable2").close(); + executeString("DROP DATABASE testcreatetable2_2").close(); + assertDatabaseNotExists("testcreatetable2"); + assertDatabaseNotExists("testcreatetable2_2"); } private final void assertPathOfCreatedTable(final String databaseName, @@ -211,13 +211,15 @@ public final void testDelimitedIdentifierWithMixedCharacters() throws Exception ResultSet res = null; try { - List tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table1", "\"TABLE1\""); + List tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table1", + "\"TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS\""); assertTableExists(tableNames.get(0)); - tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table2", "\"tablE1\""); + tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table2", + "\"testDelimitedIdentifierWithMixedCharacters\""); assertTableExists(tableNames.get(0)); - // SELECT "aGe", "tExt", "Number" FROM "TABLE1"; + // SELECT "aGe", "tExt", "Number" FROM "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS"; res = executeFile("quoted_identifier_mixed_chars_1.sql"); assertResultSet(res, "quoted_identifier_mixed_chars_1.result"); } finally { @@ -603,25 +605,25 @@ public final void testCreateTableLike1() throws Exception { @Test public final void testNestedRecord1() throws Exception { - executeString("CREATE DATABASE D9;").close(); + executeString("CREATE DATABASE d9;").close(); assertTableNotExists("d9.nested_table"); executeQuery().close(); assertTableExists("d9.nested_table"); - executeString("DROP TABLE D9.nested_table"); - executeString("DROP DATABASE D9").close(); + executeString("DROP TABLE d9.nested_table"); + executeString("DROP DATABASE d9").close(); } @Test public final void testNestedRecord2() throws Exception { - executeString("CREATE DATABASE D9;").close(); + executeString("CREATE DATABASE d10;").close(); - assertTableNotExists("d9.nested_table2"); + assertTableNotExists("d10.nested_table2"); executeQuery(); - assertTableExists("d9.nested_table2"); + assertTableExists("d10.nested_table2"); - executeString("DROP TABLE D9.nested_table2"); - executeString("DROP DATABASE D9").close(); + executeString("DROP TABLE d10.nested_table2"); + executeString("DROP DATABASE d10").close(); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java index 569111c595..0d4d2922f0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java @@ -23,7 +23,10 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.*; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter; @@ -38,8 +41,8 @@ import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.logical.ScanNode; import org.apache.tajo.storage.StorageConstants; -import org.apache.tajo.storage.TablespaceManager; import org.apache.tajo.storage.Tablespace; +import org.apache.tajo.storage.TablespaceManager; import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.storage.hbase.*; import org.apache.tajo.util.Bytes; @@ -55,10 +58,12 @@ import java.net.URI; import java.sql.ResultSet; import java.text.DecimalFormat; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import static org.junit.Assert.*; -import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) public class TestHBaseTable extends QueryTestCaseBase { @@ -97,7 +102,7 @@ public static void afterClass() { @Test public void testVerifyCreateHBaseTableRequiredMeta() throws Exception { try { - executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) TABLESPACE cluster1 USING hbase").close(); + executeString("CREATE TABLE testverifycreatehbasetablerequiredmeta (col1 text, col2 text) TABLESPACE cluster1 USING hbase").close(); fail("hbase table must have 'table' meta"); } catch (Exception e) { @@ -105,9 +110,9 @@ public void testVerifyCreateHBaseTableRequiredMeta() throws Exception { } try { - executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) TABLESPACE cluster1 " + + executeString("CREATE TABLE testverifycreatehbasetablerequiredmeta (col1 text, col2 text) TABLESPACE cluster1 " + "USING hbase " + - "WITH ('table'='hbase_table')").close(); + "WITH ('table'='testverifycreatehbasetablerequiredmeta')").close(); fail("hbase table must have 'columns' meta"); } catch (Exception e) { @@ -118,14 +123,14 @@ public void testVerifyCreateHBaseTableRequiredMeta() throws Exception { @Test public void testCreateHBaseTable() throws Exception { executeString( - "CREATE TABLE hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:a,col3:,col2:b')").close(); + "CREATE TABLE testcreatehbasetable (col1 text, col2 text, col3 text, col4 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testcreatehbasetable', 'columns'=':key,col2:a,col3:,col2:b')").close(); - assertTableExists("hbase_mapped_table1"); + assertTableExists("testcreatehbasetable"); - HTableDescriptor hTableDesc = testingCluster.getHBaseUtil().getTableDescriptor("hbase_table"); + HTableDescriptor hTableDesc = testingCluster.getHBaseUtil().getTableDescriptor("testcreatehbasetable"); assertNotNull(hTableDesc); - assertEquals("hbase_table", hTableDesc.getNameAsString()); + assertEquals("testcreatehbasetable", hTableDesc.getNameAsString()); HColumnDescriptor[] hColumns = hTableDesc.getColumnFamilies(); // col1 is mapped to rowkey @@ -133,11 +138,11 @@ public void testCreateHBaseTable() throws Exception { assertEquals("col2", hColumns[0].getNameAsString()); assertEquals("col3", hColumns[1].getNameAsString()); - executeString("DROP TABLE hbase_mapped_table1 PURGE").close(); + executeString("DROP TABLE testcreatehbasetable PURGE").close(); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); try { - assertFalse(hAdmin.tableExists("hbase_table")); + assertFalse(hAdmin.tableExists("testcreatehbasetable")); } finally { hAdmin.close(); } @@ -146,8 +151,8 @@ public void testCreateHBaseTable() throws Exception { @Test public void testCreateNotExistsExternalHBaseTable() throws Exception { String sql = String.format( - "CREATE EXTERNAL TABLE external_hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) " + - "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:a,col3:,col2:b') " + + "CREATE EXTERNAL TABLE testcreatenotexistsexternalhbasetable (col1 text, col2 text, col3 text, col4 text) " + + "USING hbase WITH ('table'='testcreatenotexistsexternalhbasetable', 'columns'=':key,col2:a,col3:,col2:b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); try { executeString(sql).close(); @@ -160,8 +165,8 @@ public void testCreateNotExistsExternalHBaseTable() throws Exception { @Test public void testCreateRowFieldWithNonText() throws Exception { try { - executeString("CREATE TABLE hbase_mapped_table2 (rk1 int4, rk2 text, col3 text, col4 text) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'='0:key#b,1:key,col3:,col2:b', " + + executeString("CREATE TABLE testcreaterowfieldwithnontext (rk1 int4, rk2 text, col3 text, col4 text) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='testcreaterowfieldwithnontext', 'columns'='0:key#b,1:key,col3:,col2:b', " + "'hbase.rowkey.delimiter'='_')").close(); fail("Key field type should be TEXT type"); } catch (Exception e) { @@ -171,27 +176,27 @@ public void testCreateRowFieldWithNonText() throws Exception { @Test public void testCreateExternalHBaseTable() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table_not_purge")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testcreateexternalhbasetable")); hTableDesc.addFamily(new HColumnDescriptor("col1")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " + - "USING hbase WITH ('table'='external_hbase_table_not_purge', 'columns'=':key,col1:a,col2:,col3:b') " + + "CREATE EXTERNAL TABLE testcreateexternalhbasetable (rk text, col1 text, col2 text, col3 text) " + + "USING hbase WITH ('table'='testcreateexternalhbasetable', 'columns'=':key,col1:a,col2:,col3:b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("external_hbase_mapped_table"); + assertTableExists("testcreateexternalhbasetable"); - executeString("DROP TABLE external_hbase_mapped_table").close(); + executeString("DROP TABLE testcreateexternalhbasetable").close(); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); try { - assertTrue(hAdmin.tableExists("external_hbase_table_not_purge")); - hAdmin.disableTable("external_hbase_table_not_purge"); - hAdmin.deleteTable("external_hbase_table_not_purge"); + assertTrue(hAdmin.tableExists("testcreateexternalhbasetable")); + hAdmin.disableTable("testcreateexternalhbasetable"); + hAdmin.deleteTable("testcreateexternalhbasetable"); } finally { hAdmin.close(); } @@ -199,23 +204,23 @@ public void testCreateExternalHBaseTable() throws Exception { @Test public void testSimpleSelectQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testsimpleselectquery")); hTableDesc.addFamily(new HColumnDescriptor("col1")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " + - "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col1:a,col2:,col3:b') " + + "CREATE EXTERNAL TABLE testsimpleselectquery (rk text, col1 text, col2 text, col3 text) " + + "USING hbase WITH ('table'='testsimpleselectquery', 'columns'=':key,col1:a,col2:,col3:b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("external_hbase_mapped_table"); + assertTableExists("testsimpleselectquery"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("external_hbase_table"); + HTableInterface htable = hconn.getTable("testsimpleselectquery"); try { for (int i = 0; i < 100; i++) { @@ -228,34 +233,34 @@ public void testSimpleSelectQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from external_hbase_mapped_table where rk > '20'"); + ResultSet res = executeString("select * from testsimpleselectquery where rk > '20'"); assertResultSet(res); cleanupQuery(res); } finally { - executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testsimpleselectquery PURGE").close(); htable.close(); } } @Test public void testBinaryMappedQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testbinarymappedquery")); hTableDesc.addFamily(new HColumnDescriptor("col1")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk int8, col1 text, col2 text, col3 int4)\n " + - "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key#b,col1:a,col2:,col3:b#b') " + + "CREATE EXTERNAL TABLE testbinarymappedquery (rk int8, col1 text, col2 text, col3 int4)\n " + + "USING hbase WITH ('table'='testbinarymappedquery', 'columns'=':key#b,col1:a,col2:,col3:b#b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("external_hbase_mapped_table"); + assertTableExists("testbinarymappedquery"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("external_hbase_table"); + HTableInterface htable = hconn.getTable("testbinarymappedquery"); try { for (int i = 0; i < 100; i++) { @@ -268,12 +273,12 @@ public void testBinaryMappedQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from external_hbase_mapped_table where rk > 20"); + ResultSet res = executeString("select * from testbinarymappedquery where rk > 20"); assertResultSet(res); res.close(); //Projection - res = executeString("select col3, col2, rk from external_hbase_mapped_table where rk > 95"); + res = executeString("select col3, col2, rk from testbinarymappedquery where rk > 95"); String expected = "col3,col2,rk\n" + "-------------------------------\n" + @@ -286,29 +291,29 @@ public void testBinaryMappedQuery() throws Exception { res.close(); } finally { - executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testbinarymappedquery PURGE").close(); htable.close(); } } @Test public void testColumnKeyValueSelectQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testcolumnkeyvalueselectquery")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, col2_key text, col2_value text, col3 text) " + - "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " + + "CREATE EXTERNAL TABLE testcolumnkeyvalueselectquery (rk1 text, col2_key text, col2_value text, col3 text) " + + "USING hbase WITH ('table'='testcolumnkeyvalueselectquery', 'columns'=':key,col2:key:,col2:value:,col3:', " + "'hbase.rowkey.delimiter'='_') LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("external_hbase_mapped_table"); + assertTableExists("testcolumnkeyvalueselectquery"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("external_hbase_table"); + HTableInterface htable = hconn.getTable("testcolumnkeyvalueselectquery"); try { for (int i = 0; i < 10; i++) { @@ -320,32 +325,32 @@ public void testColumnKeyValueSelectQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 >= 'rk-0'"); + ResultSet res = executeString("select * from testcolumnkeyvalueselectquery where rk1 >= 'rk-0'"); assertResultSet(res); cleanupQuery(res); } finally { - executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testcolumnkeyvalueselectquery PURGE").close(); htable.close(); } } @Test public void testRowFieldSelectQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testrowfieldselectquery")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, rk2 text, col3 text) " + - "USING hbase WITH ('table'='external_hbase_table', 'columns'='0:key,1:key,col3:a', " + + "CREATE EXTERNAL TABLE testrowfieldselectquery (rk1 text, rk2 text, col3 text) " + + "USING hbase WITH ('table'='testrowfieldselectquery', 'columns'='0:key,1:key,col3:a', " + "'hbase.rowkey.delimiter'='_') LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("external_hbase_mapped_table"); + assertTableExists("testrowfieldselectquery"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("external_hbase_table"); + HTableInterface htable = hconn.getTable("testrowfieldselectquery"); try { for (int i = 0; i < 100; i++) { @@ -354,11 +359,11 @@ public void testRowFieldSelectQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 > 'field1-20'"); + ResultSet res = executeString("select * from testrowfieldselectquery where rk1 > 'field1-20'"); assertResultSet(res); cleanupQuery(res); } finally { - executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testrowfieldselectquery PURGE").close(); htable.close(); } } @@ -366,17 +371,17 @@ public void testRowFieldSelectQuery() throws Exception { @Test public void testIndexPredication() throws Exception { String sql = - "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b', " + + "CREATE TABLE testindexpredication (rk text, col1 text, col2 text, col3 text) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='testindexpredication', 'columns'=':key,col1:a,col2:,col3:b', " + "'hbase.split.rowkeys'='010,040,060,080') "; executeString(sql).close(); - assertTableExists("hbase_mapped_table"); + assertTableExists("testindexpredication"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); - hAdmin.tableExists("hbase_table"); + hAdmin.tableExists("testindexpredication"); - HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testindexpredication"); try { org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -391,13 +396,13 @@ public void testIndexPredication() throws Exception { put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes()); htable.put(put); } - assertIndexPredication(false); + assertIndexPredication(false, "testindexpredication"); - ResultSet res = executeString("select * from hbase_mapped_table where rk >= '020' and rk <= '055'"); + ResultSet res = executeString("select * from testindexpredication where rk >= '020' and rk <= '055'"); assertResultSet(res); res.close(); - res = executeString("select * from hbase_mapped_table where rk = '021'"); + res = executeString("select * from testindexpredication where rk = '021'"); String expected = "rk,col1,col2,col3\n" + "-------------------------------\n" + "021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n"; @@ -405,7 +410,7 @@ public void testIndexPredication() throws Exception { assertEquals(expected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testindexpredication PURGE").close(); htable.close(); hAdmin.close(); } @@ -414,16 +419,16 @@ public void testIndexPredication() throws Exception { @Test public void testCompositeRowIndexPredication() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk text, rk2 text, col1 text, col2 text, col3 text) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + + executeString("CREATE TABLE testcompositerowindexpredication (rk text, rk2 text, col1 text, col2 text, col3 text) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='testcompositerowindexpredication', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + "'hbase.split.rowkeys'='010,040,060,080', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("hbase_mapped_table"); + assertTableExists("testcompositerowindexpredication"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); - hAdmin.tableExists("hbase_table"); + hAdmin.tableExists("testcompositerowindexpredication"); - HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testcompositerowindexpredication"); try { org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -451,9 +456,9 @@ public void testCompositeRowIndexPredication() throws Exception { assertEquals("021_021", new String(result.getRow())); scanner.close(); - assertIndexPredication(true); + assertIndexPredication(true, "testcompositerowindexpredication"); - ResultSet res = executeString("select * from hbase_mapped_table where rk = '021'"); + ResultSet res = executeString("select * from testcompositerowindexpredication where rk = '021'"); String expected = "rk,rk2,col1,col2,col3\n" + "-------------------------------\n" + "021,021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n"; @@ -461,15 +466,15 @@ public void testCompositeRowIndexPredication() throws Exception { assertEquals(expected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testcompositerowindexpredication PURGE").close(); htable.close(); hAdmin.close(); } } - private void assertIndexPredication(boolean isCompositeRowKey) throws Exception { + private void assertIndexPredication(boolean isCompositeRowKey, String table) throws Exception { String postFix = isCompositeRowKey ? "_" + new String(new char[]{Character.MAX_VALUE}) : ""; - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), table); ScanNode scanNode = new ScanNode(1); @@ -478,7 +483,7 @@ private void assertIndexPredication(boolean isCompositeRowKey) throws Exception new ConstEval(new TextDatum("021"))); scanNode.setQual(evalNodeEq); Tablespace tablespace = TablespaceManager.getByName("cluster1").get(); - List fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); + List fragments = tablespace.getSplits(table, tableDesc, scanNode); assertEquals(1, fragments.size()); assertEquals("021", new String(((HBaseFragment)fragments.get(0)).getStartRow())); assertEquals("021" + postFix, new String(((HBaseFragment)fragments.get(0)).getStopRow())); @@ -491,7 +496,7 @@ private void assertIndexPredication(boolean isCompositeRowKey) throws Exception EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2); scanNode.setQual(evalNodeA); - fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); + fragments = tablespace.getSplits(table, tableDesc, scanNode); assertEquals(2, fragments.size()); HBaseFragment fragment1 = (HBaseFragment) fragments.get(0); assertEquals("020", new String(fragment1.getStartRow())); @@ -506,7 +511,7 @@ private void assertIndexPredication(boolean isCompositeRowKey) throws Exception new ConstEval(new TextDatum("075"))); EvalNode evalNodeB = new BinaryEval(EvalType.OR, evalNodeA, evalNode3); scanNode.setQual(evalNodeB); - fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); + fragments = tablespace.getSplits(table, tableDesc, scanNode); assertEquals(3, fragments.size()); fragment1 = (HBaseFragment) fragments.get(0); assertEquals("020", new String(fragment1.getStartRow())); @@ -529,7 +534,7 @@ private void assertIndexPredication(boolean isCompositeRowKey) throws Exception EvalNode evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5); EvalNode evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC); scanNode.setQual(evalNodeD); - fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); + fragments = tablespace.getSplits(table, tableDesc, scanNode); assertEquals(3, fragments.size()); fragment1 = (HBaseFragment) fragments.get(0); @@ -552,7 +557,7 @@ private void assertIndexPredication(boolean isCompositeRowKey) throws Exception evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5); evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC); scanNode.setQual(evalNodeD); - fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); + fragments = tablespace.getSplits(table, tableDesc, scanNode); assertEquals(2, fragments.size()); fragment1 = (HBaseFragment) fragments.get(0); @@ -566,16 +571,16 @@ private void assertIndexPredication(boolean isCompositeRowKey) throws Exception @Test public void testNonForwardQuery() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:#b', " + + executeString("CREATE TABLE testnonforwardquery (rk text, col1 text, col2 text, col3 int) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='testnonforwardquery', 'columns'=':key,col1:a,col2:,col3:#b', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("hbase_mapped_table"); + assertTableExists("testnonforwardquery"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); HTable htable = null; try { - hAdmin.tableExists("hbase_table"); - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + hAdmin.tableExists("testnonforwardquery"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testnonforwardquery"); org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -590,11 +595,11 @@ public void testNonForwardQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from hbase_mapped_table"); + ResultSet res = executeString("select * from testnonforwardquery"); assertResultSet(res); res.close(); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testnonforwardquery PURGE").close(); hAdmin.close(); if (htable == null) { htable.close(); @@ -604,16 +609,16 @@ public void testNonForwardQuery() throws Exception { @Test public void testJoin() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int8) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " + + executeString("CREATE TABLE testjoin (rk text, col1 text, col2 text, col3 int8) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='testjoin', 'columns'=':key,col1:a,col2:,col3:b#b', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("hbase_mapped_table"); + assertTableExists("testjoin"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); HTable htable = null; try { - hAdmin.tableExists("hbase_table"); - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + hAdmin.tableExists("testjoin"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testjoin"); org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -629,12 +634,12 @@ public void testJoin() throws Exception { } ResultSet res = executeString("select a.rk, a.col1, a.col2, a.col3, b.l_orderkey, b.l_linestatus " + - "from hbase_mapped_table a " + + "from testjoin a " + "join default.lineitem b on a.col3 = b.l_orderkey"); assertResultSet(res); res.close(); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testjoin PURGE").close(); hAdmin.close(); if (htable != null) { htable.close(); @@ -644,19 +649,19 @@ public void testJoin() throws Exception { @Test public void testInsertInto() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); + executeString("CREATE TABLE testinsertinto (rk text, col1 text, col2 text, col3 int4) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='testinsertinto', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertinto"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertinto"); - executeString("insert into hbase_mapped_table " + + executeString("insert into testinsertinto " + "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertinto"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -670,7 +675,7 @@ public void testInsertInto() throws Exception { new boolean[]{false, false, false, true}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testinsertinto PURGE").close(); if (scanner != null) { scanner.close(); @@ -684,21 +689,21 @@ public void testInsertInto() throws Exception { @Test public void testInsertValues1() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); + executeString("CREATE TABLE testinsertvalues1 (rk text, col1 text, col2 text, col3 int4) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='testinsertvalues1', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertvalues1"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertvalues1"); - executeString("insert into hbase_mapped_table select 'aaa', 'a12', 'a34', 1").close(); - executeString("insert into hbase_mapped_table select 'bbb', 'b12', 'b34', 2").close(); - executeString("insert into hbase_mapped_table select 'ccc', 'c12', 'c34', 3").close(); - executeString("insert into hbase_mapped_table select 'ddd', 'd12', 'd34', 4").close(); + executeString("insert into testinsertvalues1 select 'aaa', 'a12', 'a34', 1").close(); + executeString("insert into testinsertvalues1 select 'bbb', 'b12', 'b34', 2").close(); + executeString("insert into testinsertvalues1 select 'ccc', 'c12', 'c34', 3").close(); + executeString("insert into testinsertvalues1 select 'ddd', 'd12', 'd34', 4").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertvalues1"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -712,7 +717,7 @@ public void testInsertValues1() throws Exception { new boolean[]{false, false, false, true}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testinsertvalues1 PURGE").close(); if (scanner != null) { scanner.close(); @@ -726,12 +731,12 @@ public void testInsertValues1() throws Exception { @Test public void testInsertIntoMultiRegion() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + + executeString("CREATE TABLE testinsertintomultiregion (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintomultiregion', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintomultiregion"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregion"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -746,16 +751,16 @@ public void testInsertIntoMultiRegion() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregion", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into hbase_mapped_table " + - "select id, name from base_table ").close(); + executeString("insert into testinsertintomultiregion " + + "select id, name from base_testinsertintomultiregion ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregion"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -767,8 +772,8 @@ public void testInsertIntoMultiRegion() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintomultiregion PURGE").close(); + executeString("DROP TABLE testinsertintomultiregion PURGE").close(); if (scanner != null) { scanner.close(); @@ -782,12 +787,12 @@ public void testInsertIntoMultiRegion() throws Exception { @Test public void testInsertIntoMultiRegion2() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + + executeString("CREATE TABLE testinsertintomultiregion2 (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintomultiregion2', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintomultiregion2"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregion2"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -801,16 +806,16 @@ public void testInsertIntoMultiRegion2() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(i + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregion2", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into hbase_mapped_table " + - "select id, name from base_table ").close(); + executeString("insert into testinsertintomultiregion2 " + + "select id, name from base_testinsertintomultiregion2 ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregion2"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -822,8 +827,8 @@ public void testInsertIntoMultiRegion2() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintomultiregion2 PURGE").close(); + executeString("DROP TABLE testinsertintomultiregion2 PURGE").close(); if (scanner != null) { scanner.close(); @@ -840,12 +845,12 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { String splitFilePath = currentDatasetPath + "/splits.data"; executeString( - "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + + "CREATE TABLE testinsertintomultiregionwithsplitfile (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintomultiregionwithsplitfile', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys.file'='" + splitFilePath + "')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintomultiregionwithsplitfile"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregionwithsplitfile"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -860,16 +865,16 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregionwithsplitfile", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into hbase_mapped_table " + - "select id, name from base_table ").close(); + executeString("insert into testinsertintomultiregionwithsplitfile " + + "select id, name from base_testinsertintomultiregionwithsplitfile ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregionwithsplitfile"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -881,8 +886,8 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintomultiregionwithsplitfile PURGE").close(); + executeString("DROP TABLE testinsertintomultiregionwithsplitfile PURGE").close(); if (scanner != null) { scanner.close(); @@ -897,13 +902,13 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { @Test public void testInsertIntoMultiRegionMultiRowFields() throws Exception { executeString( - "CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a', " + + "CREATE TABLE testinsertintomultiregionmultirowfields (rk1 text, rk2 text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintomultiregionmultirowfields', 'columns'='0:key,1:key,col1:a', " + "'hbase.split.rowkeys'='001,002,003,004,005,006,007,008,009', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintomultiregionmultirowfields"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregionmultirowfields"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -919,16 +924,16 @@ public void testInsertIntoMultiRegionMultiRowFields() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|" + (i + 100) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregionmultirowfields", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into hbase_mapped_table " + - "select id1, id2, name from base_table ").close(); + executeString("insert into testinsertintomultiregionmultirowfields " + + "select id1, id2, name from base_testinsertintomultiregionmultirowfields ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregionmultirowfields"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -940,8 +945,8 @@ public void testInsertIntoMultiRegionMultiRowFields() throws Exception { new boolean[]{false, false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintomultiregionmultirowfields PURGE").close(); + executeString("DROP TABLE testinsertintomultiregionmultirowfields PURGE").close(); if (scanner != null) { scanner.close(); @@ -955,12 +960,12 @@ public void testInsertIntoMultiRegionMultiRowFields() throws Exception { @Test public void testInsertIntoBinaryMultiRegion() throws Exception { - executeString("CREATE TABLE hbase_mapped_table (rk int4, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key#b,col1:a', " + + executeString("CREATE TABLE testinsertintobinarymultiregion (rk int4, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintobinarymultiregion', 'columns'=':key#b,col1:a', " + "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintobinarymultiregion"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintobinarymultiregion"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -974,16 +979,16 @@ public void testInsertIntoBinaryMultiRegion() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(i + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintobinarymultiregion", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into hbase_mapped_table " + - "select id, name from base_table ").close(); + executeString("insert into testinsertintobinarymultiregion " + + "select id, name from base_testinsertintobinarymultiregion ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintobinarymultiregion"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -995,8 +1000,8 @@ public void testInsertIntoBinaryMultiRegion() throws Exception { new boolean[]{true, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintobinarymultiregion PURGE").close(); + executeString("DROP TABLE testinsertintobinarymultiregion PURGE").close(); if (scanner != null) { scanner.close(); @@ -1011,12 +1016,12 @@ public void testInsertIntoBinaryMultiRegion() throws Exception { @Test public void testInsertIntoColumnKeyValue() throws Exception { executeString( - "CREATE TABLE hbase_mapped_table (rk text, col2_key text, col2_value text, col3 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " + + "CREATE TABLE testinsertintocolumnkeyvalue (rk text, col2_key text, col2_value text, col3 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintocolumnkeyvalue', 'columns'=':key,col2:key:,col2:value:,col3:', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintocolumnkeyvalue"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintocolumnkeyvalue"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -1034,16 +1039,16 @@ public void testInsertIntoColumnKeyValue() throws Exception { datas.add(i + "|ck-" + j + "|value-" + j + "|col3-" + i); } } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintocolumnkeyvalue", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into hbase_mapped_table " + - "select rk, col2_key, col2_value, col3 from base_table ").close(); + executeString("insert into testinsertintocolumnkeyvalue " + + "select rk, col2_key, col2_value, col3 from base_testinsertintocolumnkeyvalue ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintocolumnkeyvalue"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col2")); @@ -1055,7 +1060,7 @@ public void testInsertIntoColumnKeyValue() throws Exception { new byte[][]{null, null, null}, new boolean[]{false, false, false}, tableDesc.getSchema())); - ResultSet res = executeString("select * from hbase_mapped_table"); + ResultSet res = executeString("select * from testinsertintocolumnkeyvalue"); String expected = "rk,col2_key,col2_value,col3\n" + "-------------------------------\n" + @@ -1085,8 +1090,8 @@ public void testInsertIntoColumnKeyValue() throws Exception { res.close(); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintocolumnkeyvalue PURGE").close(); + executeString("DROP TABLE testinsertintocolumnkeyvalue PURGE").close(); if (scanner != null) { scanner.close(); @@ -1101,11 +1106,11 @@ public void testInsertIntoColumnKeyValue() throws Exception { @Test public void testInsertIntoDifferentType() throws Exception { executeString( - "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + + "CREATE TABLE testinsertintodifferenttype (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintodifferenttype', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close(); - assertTableExists("hbase_mapped_table"); + assertTableExists("testinsertintodifferenttype"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -1119,39 +1124,39 @@ public void testInsertIntoDifferentType() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(i + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintodifferenttype", schema, tableOptions, datas.toArray(new String[]{}), 2); try { - executeString("insert into hbase_mapped_table " + - "select id, name from base_table ").close(); + executeString("insert into testinsertintodifferenttype " + + "select id, name from base_testinsertintodifferenttype ").close(); fail("If inserting data type different with target table data type, should throw exception"); } catch (Exception e) { assertTrue(e.getMessage().indexOf("is different column type with") >= 0); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintodifferenttype PURGE").close(); + executeString("DROP TABLE testinsertintodifferenttype PURGE").close(); } } @Test public void testInsertIntoRowField() throws Exception { executeString( - "CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text, col2 text, col3 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + + "CREATE TABLE testinsertintorowfield (rk1 text, rk2 text, col1 text, col2 text, col3 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintorowfield', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintorowfield"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintorowfield"); - executeString("insert into hbase_mapped_table " + + executeString("insert into testinsertintorowfield " + "select l_orderkey::text, l_partkey::text, l_shipdate, l_returnflag, l_suppkey::text from default.lineitem "); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintorowfield"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -1165,7 +1170,7 @@ public void testInsertIntoRowField() throws Exception { new boolean[]{false, false, false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testinsertintorowfield PURGE").close(); if (scanner != null) { scanner.close(); @@ -1192,23 +1197,23 @@ public void testCTAS() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testctas", schema, tableOptions, datas.toArray(new String[]{}), 2); executeString( - "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + + "CREATE TABLE testctas (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testctas', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='010,040,060,080') as" + - " select id, name from base_table" + " select id, name from base_testctas" ).close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testctas"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testctas"); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testctas"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -1220,8 +1225,8 @@ public void testCTAS() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testctas PURGE").close(); + executeString("DROP TABLE testctas PURGE").close(); if (scanner != null) { scanner.close(); @@ -1233,9 +1238,9 @@ public void testCTAS() throws Exception { // TODO - rollback should support its corresponding hbase table HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); - if (hAdmin.tableExists("hbase_table")) { - hAdmin.disableTable("hbase_table"); - hAdmin.deleteTable("hbase_table"); + if (hAdmin.tableExists("testctas")) { + hAdmin.disableTable("testctas"); + hAdmin.deleteTable("testctas"); } } } @@ -1243,11 +1248,11 @@ public void testCTAS() throws Exception { @Test public void testInsertIntoUsingPut() throws Exception { executeString( - "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); + "CREATE TABLE testinsertintousingput (rk text, col1 text, col2 text, col3 int4) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintousingput', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); - assertTableExists("hbase_mapped_table"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); + assertTableExists("testinsertintousingput"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintousingput"); Map sessions = new HashMap(); sessions.put(HBaseStorageConstants.INSERT_PUT_MODE, "true"); @@ -1257,11 +1262,11 @@ public void testInsertIntoUsingPut() throws Exception { ResultScanner scanner = null; try { executeString( - "insert into hbase_mapped_table " + + "insert into testinsertintousingput " + "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem" ).close(); - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintousingput"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -1276,7 +1281,7 @@ public void testInsertIntoUsingPut() throws Exception { new boolean[]{false, false, false, true}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE testinsertintousingput PURGE").close(); client.unsetSessionVariables(TUtil.newList(HBaseStorageConstants.INSERT_PUT_MODE)); @@ -1293,11 +1298,11 @@ public void testInsertIntoUsingPut() throws Exception { @Test public void testInsertIntoLocation() throws Exception { executeString( - "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:', " + + "CREATE TABLE testinsertintolocation (rk text, col1 text, col2 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='testinsertintolocation', 'columns'=':key,col1:a,col2:', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("hbase_mapped_table"); + assertTableExists("testinsertintolocation"); try { // create test table @@ -1314,11 +1319,11 @@ public void testInsertIntoLocation() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i + "|comment-" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintolocation", schema, tableOptions, datas.toArray(new String[]{}), 2); executeString("insert into location '/tmp/hfile_test' " + - "select id, name, comment from base_table ").close(); + "select id, name, comment from base_testinsertintolocation ").close(); FileSystem fs = testingCluster.getDefaultFileSystem(); Path path = new Path("/tmp/hfile_test"); @@ -1338,8 +1343,8 @@ public void testInsertIntoLocation() throws Exception { index++; } } finally { - executeString("DROP TABLE base_table PURGE").close(); - executeString("DROP TABLE hbase_mapped_table PURGE").close(); + executeString("DROP TABLE base_testinsertintolocation PURGE").close(); + executeString("DROP TABLE testinsertintolocation PURGE").close(); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java index 265f075cb3..7987421467 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java +++ b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java @@ -37,6 +37,7 @@ import static junit.framework.TestCase.assertEquals; import static org.junit.Assert.assertNotEquals; +@net.jcip.annotations.NotThreadSafe public class TestHAServiceHDFSImpl { private TajoTestingCluster cluster; diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index 9b4c06997d..8a73c02aa1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -19,6 +19,7 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Lists; +import net.jcip.annotations.NotThreadSafe; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; @@ -61,6 +62,7 @@ import static org.junit.Assert.*; +@NotThreadSafe public class TestKillQuery { private static TajoTestingCluster cluster; private static TajoConf conf; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java index a91fc30513..aa63968842 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java @@ -67,7 +67,7 @@ public void tearDown(){ @Test public void testGet() throws IOException { Random rnd = new Random(); - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); String sid = "1"; String partId = "1"; @@ -116,7 +116,7 @@ public void testAdjustFetchProcess() { @Test public void testStatus() throws Exception { Random rnd = new Random(); - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); String sid = "1"; String ta = "1_0"; String partId = "1"; @@ -145,7 +145,7 @@ public void testStatus() throws Exception { @Test public void testNoContentFetch() throws Exception { - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); String sid = "1"; String ta = "1_0"; String partId = "1"; @@ -176,7 +176,7 @@ public void testNoContentFetch() throws Exception { public void testFailureStatus() throws Exception { Random rnd = new Random(); - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); String sid = "1"; String ta = "1_0"; String partId = "1"; @@ -208,7 +208,7 @@ public void testFailureStatus() throws Exception { @Test public void testServerFailure() throws Exception { - QueryId queryId = QueryIdFactory.NULL_QUERY_ID; + QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); String sid = "1"; String ta = "1_0"; String partId = "1"; diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql index 208445628b..fd45f920a7 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql @@ -1 +1 @@ -SELECT "aGe", "teXt", "Number" FROM "TABLE1"; \ No newline at end of file +SELECT "aGe", "teXt", "Number" FROM "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS"; \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql index 4b05657653..a5a28c180b 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql @@ -1 +1 @@ -SELECT "tb1"."aGe" as "AgE", "tb1"."teXt" as "TEXT", "Number" as "NUMBER" FROM "TABLE1" "tb1"; \ No newline at end of file +SELECT "tb1"."aGe" as "AgE", "tb1"."teXt" as "TEXT", "Number" as "NUMBER" FROM "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS" "tb1"; \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql index fbb00a02c4..9e86cb70f7 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql @@ -6,7 +6,7 @@ FROM ( "tb1"."teXt", "Number" FROM - "TABLE1" "tb1" + "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS" "tb1" UNION @@ -15,7 +15,7 @@ FROM ( "teXt", "Number" FROM - "tablE1" + "testDelimitedIdentifierWithMixedCharacters" ) T1 ORDER BY "aGe"; diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql b/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql index f794d21470..22d2803bcd 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql @@ -1 +1 @@ -CREATE TABLE D9.nested_table2 (f1 int, nf1 record (f1 int4, f3 double), nf2 record (f1 int4, nf1 record (f1 int4, f2 text), nf2 record (f1 int4, f2 text), f2 double), f2 text); \ No newline at end of file +CREATE TABLE D10.nested_table2 (f1 int, nf1 record (f1 int4, f3 double), nf2 record (f1 int4, nf1 record (f1 int4, f2 text), nf2 record (f1 int4, f2 text), f2 double), f2 text); \ No newline at end of file From 9a958082fb578396d4f99acd9359f4016e6bfe68 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 10 Jul 2015 20:03:12 +0900 Subject: [PATCH 42/80] change parallel fork to parallel thread --- .travis.yml | 2 +- tajo-core/pom.xml | 8 +- .../org/apache/tajo/TajoTestingCluster.java | 101 +++++++++--------- .../tajo/engine/query/TestGroupByQuery.java | 15 +-- .../tajo/engine/query/TestInnerJoinQuery.java | 1 + .../tajo/engine/query/TestInsertQuery.java | 1 + .../query/TestJoinOnPartitionedTables.java | 1 + .../testDistinctAggregationCaseByCase3.sql | 2 +- 8 files changed, 70 insertions(+), 61 deletions(-) diff --git a/.travis.yml b/.travis.yml index 61d56fbdfa..3d511e9da1 100644 --- a/.travis.yml +++ b/.travis.yml @@ -37,4 +37,4 @@ before_install: ulimit -t 514029 install: ./dev-support/travis-install-dependencies.sh script: - mvn clean install -Pparallel-test -DLOG_LEVEL=WARN -Dmaven.fork.count=2 + mvn clean install -Pparallel-test -DLOG_LEVEL=WARN -Dmaven.fork.count=3 diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 6ad6956730..7bc9950256 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -573,6 +573,7 @@ org.apache.maven.plugins maven-surefire-plugin + 2.18.1 test @@ -582,10 +583,11 @@ - ${maven.fork.count} - true + methods + 1 + ${maven.fork.count} false - -Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 + -Xms512m -Xmx1200m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 true true diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 71430a3e6d..9a252e0f11 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -47,10 +47,8 @@ import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.storage.FileTablespace; import org.apache.tajo.storage.TablespaceManager; -import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.KeyValueSet; -import org.apache.tajo.util.NetUtils; -import org.apache.tajo.util.Pair; +import org.apache.tajo.util.*; +import org.apache.tajo.util.FileUtil; import org.apache.tajo.worker.TajoWorker; import java.io.File; @@ -71,6 +69,7 @@ public class TajoTestingCluster { private MiniDFSCluster dfsCluster; private MiniCatalogServer catalogServer; private HBaseTestClusterUtil hbaseUtil; + private TajoClient tajoClient; private TajoMaster tajoMaster; private List tajoWorkers = new ArrayList(); @@ -126,8 +125,9 @@ void initPropertiesAndConfigs() { conf.setClassVar(ConfVars.LOGICAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, LogicalPlanTestRuleProvider.class); conf.setClassVar(ConfVars.GLOBAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, GlobalPlanTestRuleProvider.class); - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 2000); - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 3); + conf.setInt(ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.varname, 150); + conf.setInt(ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY.varname, 100); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 4); // Client API RPC conf.setIntVar(ConfVars.RPC_CLIENT_WORKER_THREAD_NUM, 2); @@ -452,6 +452,8 @@ public void shutdownMiniTajoCluster() { eachWorker.stopWorkerForce(); } tajoWorkers.clear(); + + FileUtil.cleanup(null, this.tajoClient); this.tajoMaster= null; } @@ -615,15 +617,19 @@ public static ResultSet run(String[] names, } } - public static TajoClient newTajoClient(TajoTestingCluster util) throws InterruptedException, IOException { - while(true) { - if(util.getMaster().isMasterRunning()) { - break; + public TajoClient getTajoClient() throws InterruptedException, IOException { + if(tajoClient == null) { + while(true) { + if(getMaster().isMasterRunning()) { + break; + } + Thread.sleep(1000); } - Thread.sleep(1000); + + this.tajoClient = new TajoClientImpl(ServiceTrackerFactory.get(getConfiguration())); } - TajoConf conf = util.getConfiguration(); - return new TajoClientImpl(ServiceTrackerFactory.get(conf)); + + return tajoClient; } public static void createTable(String tableName, Schema schema, @@ -635,47 +641,44 @@ public static void createTable(String tableName, Schema schema, KeyValueSet tableOption, String[] tableDatas, int numDataFiles) throws Exception { TpchTestBase instance = TpchTestBase.getInstance(); TajoTestingCluster util = instance.getTestingCluster(); - TajoClient client = newTajoClient(util); - try { - FileSystem fs = util.getDefaultFileSystem(); - Path rootDir = TajoConf.getWarehouseDir(util.getConfiguration()); - if (!fs.exists(rootDir)) { - fs.mkdirs(rootDir); - } - Path tablePath; - if (CatalogUtil.isFQTableName(tableName)) { - Pair name = CatalogUtil.separateQualifierAndName(tableName); - tablePath = new Path(rootDir, new Path(name.getFirst(), name.getSecond())); - } else { - tablePath = new Path(rootDir, tableName); - } + TajoClient client = util.getTajoClient(); + FileSystem fs = util.getDefaultFileSystem(); + Path rootDir = TajoConf.getWarehouseDir(util.getConfiguration()); - fs.mkdirs(tablePath); - if (tableDatas.length > 0) { - int recordPerFile = tableDatas.length / numDataFiles; - if (recordPerFile == 0) { - recordPerFile = 1; - } - FSDataOutputStream out = null; - for (int j = 0; j < tableDatas.length; j++) { - if (out == null || j % recordPerFile == 0) { - if (out != null) { - out.close(); - } - Path dfsPath = new Path(tablePath, tableName + j + ".tbl"); - out = fs.create(dfsPath); + if (!fs.exists(rootDir)) { + fs.mkdirs(rootDir); + } + Path tablePath; + if (CatalogUtil.isFQTableName(tableName)) { + Pair name = CatalogUtil.separateQualifierAndName(tableName); + tablePath = new Path(rootDir, new Path(name.getFirst(), name.getSecond())); + } else { + tablePath = new Path(rootDir, tableName); + } + + fs.mkdirs(tablePath); + if (tableDatas.length > 0) { + int recordPerFile = tableDatas.length / numDataFiles; + if (recordPerFile == 0) { + recordPerFile = 1; + } + FSDataOutputStream out = null; + for (int j = 0; j < tableDatas.length; j++) { + if (out == null || j % recordPerFile == 0) { + if (out != null) { + out.close(); } - out.write((tableDatas[j] + "\n").getBytes()); - } - if (out != null) { - out.close(); + Path dfsPath = new Path(tablePath, tableName + j + ".tbl"); + out = fs.create(dfsPath); } + out.write((tableDatas[j] + "\n").getBytes()); + } + if (out != null) { + out.close(); } - TableMeta meta = CatalogUtil.newTableMeta("CSV", tableOption); - client.createExternalTable(tableName, schema, tablePath.toUri(), meta); - } finally { - client.close(); } + TableMeta meta = CatalogUtil.newTableMeta("CSV", tableOption); + client.createExternalTable(tableName, schema, tablePath.toUri(), meta); } /** diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java index 305536281d..345ceaee7a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java @@ -46,6 +46,7 @@ @Category(IntegrationTest.class) @RunWith(Parameterized.class) +@net.jcip.annotations.NotThreadSafe public class TestGroupByQuery extends QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(TestGroupByQuery.class); @@ -432,12 +433,12 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { schema.addColumn("qty", Type.INT4); schema.addColumn("qty2", Type.FLOAT8); String[] data = new String[]{"1|a|3|3.0", "1|a|4|4.0", "1|b|5|5.0", "2|a|1|6.0", "2|c|2|7.0", "2|d|3|8.0"}; - TajoTestingCluster.createTable("table10", schema, tableOptions, data); + TajoTestingCluster.createTable("testdistinctaggregationcasebycase11", schema, tableOptions, data); res = executeString("select id, count(distinct code), " + "avg(qty), min(qty), max(qty), sum(qty), " + "cast(avg(qty2) as INT8), cast(min(qty2) as INT8), cast(max(qty2) as INT8), cast(sum(qty2) as INT8) " + - "from table10 group by id"); + "from testdistinctaggregationcasebycase11 group by id"); String expected = "id,?count_4,?avg_5,?min_6,?max_7,?sum_8,?cast_9,?cast_10,?cast_11,?cast_12\n" + "-------------------------------\n" + @@ -448,7 +449,7 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { // multiple distinct with expression res = executeString( - "select count(distinct code) + count(distinct qty) from table10" + "select count(distinct code) + count(distinct qty) from testdistinctaggregationcasebycase11" ); expected = "?plus_2\n" + @@ -459,7 +460,7 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { res.close(); res = executeString( - "select id, count(distinct code) + count(distinct qty) from table10 group by id" + "select id, count(distinct code) + count(distinct qty) from testdistinctaggregationcasebycase11 group by id" ); expected = "id,?plus_2\n" + @@ -470,7 +471,7 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { assertEquals(expected, resultSetToString(res)); res.close(); - executeString("DROP TABLE table10 PURGE").close(); + executeString("DROP TABLE testdistinctaggregationcasebycase11 PURGE").close(); } @Test @@ -494,13 +495,13 @@ public final void testDistinctAggregationCaseByCase3() throws Exception { "a|b-3|\\N" }; - TajoTestingCluster.createTable("table10", schema, tableOptions, data); + TajoTestingCluster.createTable("testdistinctaggregationcasebycase3", schema, tableOptions, data); ResultSet res = executeQuery(); assertResultSet(res); cleanupQuery(res); - executeString("DROP TABLE table10 PURGE").close(); + executeString("DROP TABLE testdistinctaggregationcasebycase3 PURGE").close(); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java index 6eedb4223a..77473d0774 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java @@ -33,6 +33,7 @@ @Category(IntegrationTest.class) @RunWith(Parameterized.class) @NamedTest("TestJoinQuery") +@net.jcip.annotations.NotThreadSafe public class TestInnerJoinQuery extends TestJoinQuery { public TestInnerJoinQuery(String joinOption) throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java index 7a671d8386..cb156d0e9c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java @@ -41,6 +41,7 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestInsertQuery extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java index da0f59d29d..d769c13dc7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java @@ -46,6 +46,7 @@ @Category(IntegrationTest.class) @RunWith(Parameterized.class) @NamedTest("TestJoinQuery") +@net.jcip.annotations.NotThreadSafe public class TestJoinOnPartitionedTables extends TestJoinQuery { public TestJoinOnPartitionedTables(String joinOption) throws Exception { diff --git a/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql b/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql index 925cadd21c..76b3fe36a8 100644 --- a/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql +++ b/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql @@ -3,6 +3,6 @@ select count(distinct col2) as cnt1, count(distinct case when col3 is not null then col2 else null end) as cnt2 from - table10 + testdistinctaggregationcasebycase3 group by col1; \ No newline at end of file From 81bb329fe1b30ac885ebe822d165d9ac49d3707e Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 10 Jul 2015 20:04:56 +0900 Subject: [PATCH 43/80] add NotThreadSafe annotation --- .../java/org/apache/tajo/engine/planner/TestLogicalPlanner.java | 1 + 1 file changed, 1 insertion(+) diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java index 0f377630de..4e673e8bb8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java @@ -60,6 +60,7 @@ import static org.junit.Assert.*; import static org.junit.Assert.assertEquals; +@net.jcip.annotations.NotThreadSafe public class TestLogicalPlanner { private static TajoTestingCluster util; private static CatalogService catalog; From 1d8c1ce93440c7bbf7d8404e45b47003bccaeeb8 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 10 Jul 2015 20:41:06 +0900 Subject: [PATCH 44/80] decrease thread count in test-conf --- .travis.yml | 4 ++-- .../src/test/java/org/apache/tajo/TajoTestingCluster.java | 3 +++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 3d511e9da1..39da250f54 100644 --- a/.travis.yml +++ b/.travis.yml @@ -33,8 +33,8 @@ notifications: - issues@tajo.apache.org irc: "chat.freenode.net#tajo" -before_install: ulimit -t 514029 +before_install: ulimit -t 514029 -n 3000 install: ./dev-support/travis-install-dependencies.sh script: - mvn clean install -Pparallel-test -DLOG_LEVEL=WARN -Dmaven.fork.count=3 + mvn clean install -Pparallel-test -DLOG_LEVEL=WARN -Dmaven.fork.count=2 diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 9a252e0f11..70e3a208e2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -125,9 +125,11 @@ void initPropertiesAndConfigs() { conf.setClassVar(ConfVars.LOGICAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, LogicalPlanTestRuleProvider.class); conf.setClassVar(ConfVars.GLOBAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, GlobalPlanTestRuleProvider.class); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES.varname, 4); conf.setInt(ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.varname, 150); conf.setInt(ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY.varname, 100); conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 4); + conf.setInt(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM.varname, 2); // Client API RPC conf.setIntVar(ConfVars.RPC_CLIENT_WORKER_THREAD_NUM, 2); @@ -135,6 +137,7 @@ void initPropertiesAndConfigs() { //Client API service RPC Server conf.setIntVar(ConfVars.MASTER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM, 2); conf.setIntVar(ConfVars.WORKER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM, 2); + conf.setIntVar(ConfVars.REST_SERVICE_RPC_SERVER_WORKER_THREAD_NUM, 2); // Internal RPC Client conf.setIntVar(ConfVars.INTERNAL_RPC_CLIENT_WORKER_THREAD_NUM, 2); From 7957330c1e674ca3a559bc4dd8cf483bd9482a71 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 10 Jul 2015 21:23:47 +0900 Subject: [PATCH 45/80] revert parallel-thread mode --- tajo-core/pom.xml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 7bc9950256..45243d6e1f 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -583,9 +583,8 @@ - methods - 1 - ${maven.fork.count} + ${maven.fork.count} + true false -Xms512m -Xmx1200m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 true From 1bc8f3a16e1c842ce271a4bc19091014ba74f99c Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sat, 11 Jul 2015 00:21:09 +0900 Subject: [PATCH 46/80] remove hardcoded plugin version --- tajo-core/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 45243d6e1f..54a980475e 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -573,7 +573,6 @@ org.apache.maven.plugins maven-surefire-plugin - 2.18.1 test From ec440b0d0d9e17e03809c9a8047a1d381868b726 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sat, 11 Jul 2015 01:36:44 +0900 Subject: [PATCH 47/80] print ulimit --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 39da250f54..7eb94a3b84 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,4 +1,5 @@ # 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 @@ -34,6 +35,7 @@ notifications: irc: "chat.freenode.net#tajo" before_install: ulimit -t 514029 -n 3000 +before_install: ulimit -a install: ./dev-support/travis-install-dependencies.sh script: From 9e5a308d9df435fe37f6a373913630323f3e8832 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sat, 11 Jul 2015 14:45:00 +0900 Subject: [PATCH 48/80] add timeout --- .../apache/tajo/querymaster/DefaultTaskScheduler.java | 9 ++++++--- .../java/org/apache/tajo/worker/NodeStatusUpdater.java | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 308747aec0..a448fbd5ce 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -898,7 +898,8 @@ public void assignToLeafTasks(LinkedList taskRequests) { if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); AsyncRpcClient tajoWorkerRpc = null; - CallFuture callFuture = new CallFuture(); + CallFuture + callFuture = new CallFuture(); try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); @@ -996,7 +997,8 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); - CallFuture callFuture = new CallFuture(); + CallFuture + callFuture = new CallFuture(); InetSocketAddress addr = stage.getAssignedWorkerMap().get(connectionInfo.getId()); if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); @@ -1007,7 +1009,8 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); - TajoWorkerProtocol.BatchAllocationResponseProto responseProto = callFuture.get(); + TajoWorkerProtocol.BatchAllocationResponseProto + responseProto = callFuture.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); if(responseProto.getCancellationTaskCount() > 0) { for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 8893384ec9..cb8a49a6d6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -104,7 +104,6 @@ public void serviceStop() throws Exception { this.isStopped = true; synchronized (updaterThread) { updaterThread.interrupt(); - updaterThread.join(100); } super.serviceStop(); LOG.info("NodeStatusUpdater stopped."); @@ -255,9 +254,10 @@ public void run() { LOG.fatal(cnfe.getMessage(), cnfe); Runtime.getRuntime().halt(-1); } catch (Exception e) { - LOG.error(e.getMessage(), e); if (isStopped) { break; + } else { + LOG.error(e.getMessage(), e); } } } From 18207d29c7ff3061ad91bb96b584fcfd3b7f069e Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sat, 11 Jul 2015 14:48:18 +0900 Subject: [PATCH 49/80] add NotThreadSafe annotation --- .travis.yml | 1 - tajo-core/pom.xml | 7 ++- .../org/apache/tajo/QueryTestCaseBase.java | 18 ++----- .../org/apache/tajo/benchmark/TestTPCH.java | 3 +- .../apache/tajo/cli/tools/TestTajoDump.java | 1 + .../apache/tajo/client/TestTajoClient.java | 48 ++++++++----------- .../engine/function/TestBuiltinFunctions.java | 1 + .../engine/planner/TestQueryValidation.java | 1 + .../tajo/engine/query/TestAlterTable.java | 1 + .../engine/query/TestAlterTablespace.java | 1 + .../tajo/engine/query/TestCTASQuery.java | 1 + .../tajo/engine/query/TestCaseByCases.java | 1 + .../tajo/engine/query/TestCreateDatabase.java | 1 + .../tajo/engine/query/TestCreateTable.java | 1 + .../tajo/engine/query/TestDropTable.java | 1 + .../tajo/engine/query/TestHBaseTable.java | 1 + .../query/TestJoinOnPartitionedTables.java | 1 - .../tajo/engine/query/TestJoinQuery.java | 1 + .../tajo/engine/query/TestNetTypes.java | 1 + .../engine/query/TestSelectNestedRecord.java | 1 + .../tajo/engine/query/TestSelectQuery.java | 1 + .../engine/query/TestSetSessionQuery.java | 6 +-- .../tajo/engine/query/TestSortQuery.java | 3 +- .../engine/query/TestTablePartitions.java | 1 + .../tajo/engine/query/TestTableSubQuery.java | 1 + .../tajo/engine/query/TestTruncateTable.java | 1 + .../tajo/engine/query/TestUnionQuery.java | 1 + .../tajo/engine/query/TestWindowQuery.java | 1 + .../tajo/jdbc/TestTajoDatabaseMetaData.java | 1 + .../org/apache/tajo/jdbc/TestTajoJdbc.java | 1 + ...estNonForwardQueryResultSystemScanner.java | 43 +---------------- .../tajo/master/rule/TestMasterRules.java | 15 +++--- .../querymaster/TestTaskStatusUpdate.java | 1 + .../util/history/TestHistoryWriterReader.java | 1 + .../tajo/worker/TestNodeResourceManager.java | 2 + .../tajo/worker/TestNodeStatusUpdater.java | 1 + .../apache/tajo/worker/TestTaskExecutor.java | 1 + .../apache/tajo/worker/TestTaskManager.java | 1 + .../ws/rs/resources/TestClusterResource.java | 17 ++++--- .../rs/resources/TestDatabasesResource.java | 24 +++++----- .../rs/resources/TestFunctionsResource.java | 1 + .../ws/rs/resources/TestQueryResource.java | 1 + .../rs/resources/TestQueryResultResource.java | 1 + .../ws/rs/resources/TestSessionsResource.java | 24 +++++----- .../ws/rs/resources/TestTablesResource.java | 1 + 45 files changed, 108 insertions(+), 135 deletions(-) diff --git a/.travis.yml b/.travis.yml index 7eb94a3b84..dd8485674e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -35,7 +35,6 @@ notifications: irc: "chat.freenode.net#tajo" before_install: ulimit -t 514029 -n 3000 -before_install: ulimit -a install: ./dev-support/travis-install-dependencies.sh script: diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 54a980475e..6e4a33fb55 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -150,7 +150,6 @@ org.codehaus.mojo exec-maven-plugin - 1.2 generate-sources @@ -573,6 +572,7 @@ org.apache.maven.plugins maven-surefire-plugin + 2.18.1 test @@ -584,8 +584,11 @@ ${maven.fork.count} true + classesAndMethods + 1 + ${maven.fork.count} false - -Xms512m -Xmx1200m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 + -Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 true true diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java index a323f252f6..f873c3a70a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java @@ -57,22 +57,13 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.lang.annotation.Annotation; -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; +import java.lang.annotation.*; import java.lang.reflect.Method; import java.net.URL; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; +import java.util.*; import static org.junit.Assert.*; @@ -144,7 +135,8 @@ *
  • Each query file must have the file extension '.sql'.
  • * */ -public class QueryTestCaseBase { +@net.jcip.annotations.NotThreadSafe +public abstract class QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(QueryTestCaseBase.class); protected static final TpchTestBase testBase; protected static final TajoTestingCluster testingCluster; @@ -231,7 +223,7 @@ public void printTestName() { /* protect a travis stalled build */ System.out.println("Run: " + name.getMethodName() + " Used memory: " + ((Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) - / (1024 * 1024)) + "MBytes"); + / (1024 * 1024)) + " MBytes, Running threads:" + Thread.activeCount()); } public QueryTestCaseBase() { diff --git a/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java index 9b7db2de0b..d171be125d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java +++ b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java @@ -24,9 +24,8 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -import java.sql.ResultSet; - @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestTPCH extends QueryTestCaseBase { public TestTPCH() { diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java b/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java index b371be29e8..8d73ad08aa 100644 --- a/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java +++ b/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java @@ -25,6 +25,7 @@ import java.io.ByteArrayOutputStream; import java.io.PrintWriter; +@net.jcip.annotations.NotThreadSafe public class TestTajoDump extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java index 73b97fa37f..228f0be80e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java +++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java @@ -22,7 +22,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.protobuf.ServiceException; -import net.jcip.annotations.NotThreadSafe; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileStatus; @@ -42,9 +41,7 @@ import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.storage.StorageUtil; import org.apache.tajo.util.CommonTestingUtil; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.*; import org.junit.experimental.categories.Category; import java.io.IOException; @@ -56,24 +53,33 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) -@NotThreadSafe public class TestTajoClient { private static TajoTestingCluster cluster; private static TajoConf conf; - private static TajoClient client; private static Path testDir; + private TajoClient client; @BeforeClass - public static void setUp() throws Exception { + public static void setUpClass() throws Exception { cluster = TpchTestBase.getInstance().getTestingCluster(); conf = cluster.getConfiguration(); - client = cluster.newTajoClient(); testDir = CommonTestingUtil.getTestDir(); } @AfterClass - public static void tearDown() throws Exception { + public static void tearDownClass() throws Exception { + } + + @Before + public void setUp() throws Exception { + assertNull(client); + client = cluster.newTajoClient(); + } + + @After + public void tearDown() throws Exception { client.close(); + client = null; } private static Path writeTmpTable(String tableName) throws IOException { @@ -84,20 +90,14 @@ private static Path writeTmpTable(String tableName) throws IOException { @Test public final void testCreateAndDropDatabases() throws ServiceException { - int currentNum = client.getAllDatabaseNames().size(); - String prefix = CatalogUtil.normalizeIdentifier("testCreateDatabase_"); for (int i = 0; i < 10; i++) { - // test allDatabaseNames - assertEquals(currentNum + i, client.getAllDatabaseNames().size()); - // test existence assertFalse(client.existDatabase(prefix + i)); assertTrue(client.createDatabase(prefix + i)); assertTrue(client.existDatabase(prefix + i)); // test allDatabaseNames - assertEquals(currentNum + i + 1, client.getAllDatabaseNames().size()); assertTrue(client.getAllDatabaseNames().contains(prefix + i)); } @@ -109,30 +109,25 @@ public final void testCreateAndDropDatabases() throws ServiceException { assertFalse(client.existDatabase(prefix + i)); assertFalse(client.getAllDatabaseNames().contains(prefix + i)); } - - assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test public final void testCurrentDatabase() throws IOException, ServiceException, InterruptedException { - int currentNum = client.getAllDatabaseNames().size(); + assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase()); String databaseName = CatalogUtil.normalizeIdentifier("testcurrentdatabase"); assertTrue(client.createDatabase(databaseName)); - assertEquals(currentNum + 1, client.getAllDatabaseNames().size()); + assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase()); assertTrue(client.selectDatabase(databaseName)); assertEquals(databaseName, client.getCurrentDatabase()); assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME)); assertTrue(client.dropDatabase(databaseName)); - - assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test public final void testSelectDatabaseToInvalidOne() throws IOException, ServiceException, InterruptedException { - int currentNum = client.getAllDatabaseNames().size(); assertFalse(client.existDatabase("invaliddatabase")); try { @@ -141,13 +136,12 @@ public final void testSelectDatabaseToInvalidOne() throws IOException, ServiceEx } catch (Throwable t) { assertFalse(false); } - - assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test public final void testDropCurrentDatabase() throws IOException, ServiceException, InterruptedException { - int currentNum = client.getAllDatabaseNames().size(); + assertTrue(0 < client.getAllDatabaseNames().size()); + String databaseName = CatalogUtil.normalizeIdentifier("testdropcurrentdatabase"); assertTrue(client.createDatabase(databaseName)); assertTrue(client.selectDatabase(databaseName)); @@ -162,7 +156,6 @@ public final void testDropCurrentDatabase() throws IOException, ServiceException assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME)); assertTrue(client.dropDatabase(databaseName)); - assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test @@ -620,8 +613,7 @@ public final void testGetFinishedQueryList() throws IOException, resultSet = client.executeQueryAndGetResult("select * from " + tableName + " order by deptname"); assertNotNull(resultSet); - assertEquals(numFinishedQueries + 2, client.getFinishedQueryList().size()); - + assertTrue(numFinishedQueries + 2 <= client.getFinishedQueryList().size()); resultSet.close(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java index 807ea6f78f..c556b5cf92 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java @@ -34,6 +34,7 @@ import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestBuiltinFunctions extends QueryTestCaseBase { public TestBuiltinFunctions() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java index fd60a5e53f..12d227d59e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java @@ -24,6 +24,7 @@ import java.io.IOException; +@net.jcip.annotations.NotThreadSafe public class TestQueryValidation extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java index 44fa1f3594..9a56efc1d2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java @@ -27,6 +27,7 @@ import java.util.List; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestAlterTable extends QueryTestCaseBase { @Test public final void testAlterTableName() throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java index b0ec92d2e5..32c1f49893 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java @@ -27,6 +27,7 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestAlterTablespace extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java index 2758d7d1f9..71ff04992e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java @@ -48,6 +48,7 @@ * Test CREATE TABLE AS SELECT statements */ @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestCTASQuery extends QueryTestCaseBase { public TestCTASQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java index bcf00f88eb..a2f30eb785 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java @@ -26,6 +26,7 @@ import static org.junit.Assert.assertEquals; +@net.jcip.annotations.NotThreadSafe public class TestCaseByCases extends QueryTestCaseBase { public TestCaseByCases() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java index 453c174ca3..78a0c264ff 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java @@ -27,6 +27,7 @@ import java.sql.ResultSet; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestCreateDatabase extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java index 906d8a8b59..86912fd12c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java @@ -36,6 +36,7 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestCreateTable extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java index 00201568e8..f57a850f26 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java @@ -26,6 +26,7 @@ import java.util.List; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestDropTable extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java index 0d4d2922f0..04d64db819 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java @@ -66,6 +66,7 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestHBaseTable extends QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(TestHBaseTable.class); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java index d769c13dc7..da0f59d29d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java @@ -46,7 +46,6 @@ @Category(IntegrationTest.class) @RunWith(Parameterized.class) @NamedTest("TestJoinQuery") -@net.jcip.annotations.NotThreadSafe public class TestJoinOnPartitionedTables extends TestJoinQuery { public TestJoinOnPartitionedTables(String joinOption) throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java index dd67e06496..2b8f009daf 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java @@ -51,6 +51,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +@net.jcip.annotations.NotThreadSafe public class TestJoinQuery extends QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(TestJoinQuery.class); private static int reference = 0; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java index bd8f83049b..61075cdb62 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java @@ -24,6 +24,7 @@ import java.sql.ResultSet; +@net.jcip.annotations.NotThreadSafe public class TestNetTypes extends QueryTestCaseBase { @Before diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java index 9f8a5fdbd7..8499b3f9b6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java @@ -27,6 +27,7 @@ import static org.junit.Assert.assertEquals; +@net.jcip.annotations.NotThreadSafe public class TestSelectNestedRecord extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 13515057b0..cc21c73c81 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -48,6 +48,7 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestSelectQuery extends QueryTestCaseBase { public TestSelectQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java index c9c73391b9..bd17daf735 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java @@ -21,15 +21,13 @@ import org.apache.tajo.IntegrationTest; import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; -import org.apache.tajo.catalog.CatalogUtil; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestSetSessionQuery extends QueryTestCaseBase { public TestSetSessionQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java index 24b525d8fb..cce7168dc1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java @@ -24,7 +24,6 @@ import org.apache.tajo.TajoTestingCluster; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes.Type; -import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.util.KeyValueSet; @@ -32,11 +31,11 @@ import org.junit.experimental.categories.Category; import java.sql.ResultSet; -import java.util.TimeZone; import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestSortQuery extends QueryTestCaseBase { public TestSortQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java index ef57356c80..888b8abe4e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java @@ -59,6 +59,7 @@ import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) +@net.jcip.annotations.NotThreadSafe public class TestTablePartitions extends QueryTestCaseBase { private NodeType nodeType; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java index bc643aba03..dea5a4bb66 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java @@ -24,6 +24,7 @@ import java.sql.ResultSet; +@net.jcip.annotations.NotThreadSafe public class TestTableSubQuery extends QueryTestCaseBase { public TestTableSubQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java index 1be21e4486..6ccefb14ab 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java @@ -31,6 +31,7 @@ import static org.junit.Assert.fail; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestTruncateTable extends QueryTestCaseBase { public TestTruncateTable() { super(TajoConstants.DEFAULT_DATABASE_NAME); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java index 03c9c24b01..d481d4deb6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java @@ -37,6 +37,7 @@ * - O - order by */ @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestUnionQuery extends QueryTestCaseBase { public TestUnionQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java index 9993992aa9..e6df40aa6b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java @@ -34,6 +34,7 @@ import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestWindowQuery extends QueryTestCaseBase { public TestWindowQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java index 8ee6755e11..81cdbb81d2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java +++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java @@ -33,6 +33,7 @@ import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestTajoDatabaseMetaData extends QueryTestCaseBase { private static InetSocketAddress tajoMasterAddress; diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java index ad74046712..67263bb50e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java +++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java @@ -38,6 +38,7 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestTajoJdbc extends QueryTestCaseBase { private static InetSocketAddress tajoMasterAddress; diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java index e8d59d0d5e..ac1a85a9a4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java @@ -18,49 +18,10 @@ package org.apache.tajo.master; -import static org.junit.Assert.*; -import static org.hamcrest.CoreMatchers.*; - -import java.io.File; -import java.sql.ResultSet; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.UUID; - -import org.apache.tajo.*; -import org.apache.tajo.algebra.Expr; -import org.apache.tajo.benchmark.TPCH; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.client.ResultSetUtil; -import org.apache.tajo.client.TajoClient; -import org.apache.tajo.common.TajoDataTypes.Type; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.master.TajoMaster.MasterContext; -import org.apache.tajo.master.exec.NonForwardQueryResultScanner; -import org.apache.tajo.master.exec.NonForwardQueryResultSystemScanner; -import org.apache.tajo.plan.LogicalOptimizer; -import org.apache.tajo.plan.LogicalPlan; -import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.logical.LimitNode; -import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.storage.RowStoreUtil; -import org.apache.tajo.storage.StorageConstants; -import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder; -import org.apache.tajo.storage.Tuple; -import org.apache.tajo.util.KeyValueSet; -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeDiagnosingMatcher; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.apache.tajo.QueryTestCaseBase; import org.junit.Test; -import com.google.protobuf.ByteString; - +@net.jcip.annotations.NotThreadSafe public class TestNonForwardQueryResultSystemScanner extends QueryTestCaseBase { @Test public void testGetNextRowsForAggregateFunction() throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java b/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java index dc3462065d..6a9cf59c50 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java @@ -18,9 +18,6 @@ package org.apache.tajo.master.rule; -import static org.junit.Assert.*; -import static org.hamcrest.CoreMatchers.*; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -28,18 +25,20 @@ import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.rule.EvaluationContext; -import org.apache.tajo.rule.EvaluationFailedException; -import org.apache.tajo.rule.EvaluationResult; +import org.apache.tajo.rule.*; import org.apache.tajo.rule.EvaluationResult.EvaluationResultCode; -import org.apache.tajo.rule.SelfDiagnosisRuleEngine; -import org.apache.tajo.rule.SelfDiagnosisRuleSession; import org.apache.tajo.rule.base.TajoConfValidationRule; import org.apache.tajo.util.CommonTestingUtil; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +@net.jcip.annotations.NotThreadSafe public class TestMasterRules { private static Path rootFilePath; diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java index ab5375c6d1..921fc1a11a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java @@ -36,6 +36,7 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@net.jcip.annotations.NotThreadSafe public class TestTaskStatusUpdate extends QueryTestCaseBase { public TestTaskStatusUpdate() { diff --git a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java index f442bde55a..cb2b9eecc5 100644 --- a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java +++ b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java @@ -42,6 +42,7 @@ import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestHistoryWriterReader extends QueryTestCaseBase { public static final String HISTORY_DIR = "/tmp/tajo-test-history"; TajoConf tajoConf; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 947773f72c..ea30e614a8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -42,6 +42,8 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; import static org.junit.Assert.*; + +@net.jcip.annotations.NotThreadSafe public class TestNodeResourceManager { private MockNodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 51a7771c8b..8e1e537f64 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -35,6 +35,7 @@ import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestNodeStatusUpdater { private NodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index ca19f6f5f9..d696d8ecf5 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -45,6 +45,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +@net.jcip.annotations.NotThreadSafe public class TestTaskExecutor { private NodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index 73474cfcee..c479af09fd 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -44,6 +44,7 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestTaskManager { private NodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java index c572eee252..2b7d37acb4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java @@ -18,15 +18,6 @@ package org.apache.tajo.ws.rs.resources; -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.logging.Logger; - -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.core.GenericType; - import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf.ConfVars; @@ -37,8 +28,16 @@ import org.junit.Before; import org.junit.Test; +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.core.GenericType; +import java.net.URI; +import java.util.List; +import java.util.Map; + import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestClusterResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java index 59c53f1766..698ce090b4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java @@ -18,18 +18,6 @@ package org.apache.tajo.ws.rs.resources; -import java.net.URI; -import java.util.Collection; -import java.util.Map; - -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.client.Entity; -import javax.ws.rs.core.GenericType; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; - import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf.ConfVars; @@ -42,8 +30,20 @@ import org.junit.Before; import org.junit.Test; +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.GenericType; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; +import java.net.URI; +import java.util.Collection; +import java.util.Map; + import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestDatabasesResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java index 7c7aeaed83..f1346da901 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java @@ -38,6 +38,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +@net.jcip.annotations.NotThreadSafe public class TestFunctionsResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java index 5d0b150c72..f2f8d8213d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java @@ -47,6 +47,7 @@ import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestQueryResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java index fec162688c..51ba5d3951 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java @@ -54,6 +54,7 @@ import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestQueryResultResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java index 7b293d840e..43e88b8a7e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java @@ -18,18 +18,6 @@ package org.apache.tajo.ws.rs.resources; -import java.net.URI; -import java.util.HashMap; -import java.util.Map; - -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.client.Entity; -import javax.ws.rs.core.GenericType; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; - import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf.ConfVars; @@ -42,8 +30,20 @@ import org.junit.Before; import org.junit.Test; +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.GenericType; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; + import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestSessionsResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java index 3f2bd505fe..7f9b7642e8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java @@ -44,6 +44,7 @@ import static org.junit.Assert.*; +@net.jcip.annotations.NotThreadSafe public class TestTablesResource extends QueryTestCaseBase { private URI restServiceURI; From 9d9f24e0d26359e5bf9a8f5eefdb1da07cfdedb6 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sun, 12 Jul 2015 21:42:05 +0900 Subject: [PATCH 50/80] Revert "refactor same table name for parallel testing" This reverts commit cdeee9b4fc7cbb0ce0365ce178a5b23a71f45cea. --- .../engine/codegen/TestEvalCodeGenerator.java | 326 ++--- .../tajo/engine/eval/TestPredicates.java | 134 +- .../tajo/engine/eval/TestSQLExpression.java | 1198 ++++++++--------- .../engine/function/TestBuiltinFunctions.java | 148 +- .../function/TestConditionalExpressions.java | 84 +- .../function/TestDateTimeFunctions.java | 57 +- .../engine/function/TestMathFunctions.java | 56 +- .../TestPatternMatchingPredicates.java | 30 +- .../TestStringOperatorsAndFunctions.java | 110 +- .../planner/physical/TestBNLJoinExec.java | 66 +- .../physical/TestFullOuterHashJoinExec.java | 88 +- .../physical/TestFullOuterMergeJoinExec.java | 94 +- .../physical/TestLeftOuterHashJoinExec.java | 87 +- .../physical/TestRightOuterHashJoinExec.java | 73 +- .../physical/TestRightOuterMergeJoinExec.java | 68 +- .../tajo/engine/query/TestCreateTable.java | 82 +- .../tajo/engine/query/TestHBaseTable.java | 429 +++--- .../apache/tajo/ha/TestHAServiceHDFSImpl.java | 1 - .../tajo/querymaster/TestKillQuery.java | 2 - .../org/apache/tajo/worker/TestFetcher.java | 10 +- .../quoted_identifier_mixed_chars_1.sql | 2 +- .../quoted_identifier_mixed_chars_2.sql | 2 +- .../quoted_identifier_mixed_chars_3.sql | 4 +- .../TestCreateTable/testNestedRecord2.sql | 2 +- 24 files changed, 1554 insertions(+), 1599 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java b/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java index 279c118003..d86081af8a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java @@ -45,22 +45,22 @@ public class TestEvalCodeGenerator extends ExprTestBase { @Test public void testArithmetic() throws IOException { - testEval(schema, "testarithmetic", "0,1,2,3,4.5,6.5", "select 1+1;", new String [] {"2"}); - testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col2 from testarithmetic;", new String [] {"3"}); - testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col3 from testarithmetic;", new String [] {"4"}); - testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col4 from testarithmetic;", new String [] {"5.5"}); - testEval(schema, "testarithmetic", "0,1,2,3,4.5,5.5", "select col1 + col5 from testarithmetic;", new String [] {"6.5"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1+1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col2 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col3 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col4 from table1;", new String [] {"5.5"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col5 from table1;", new String [] {"6.5"}); } @Test public void testGetField() throws IOException { - testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col1 from testgetfield;", new String [] {"1"}); - testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col2 from testgetfield;", new String [] {"2"}); - testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col3 from testgetfield;", new String [] {"3"}); - testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col4 from testgetfield;", new String [] {"4.5"}); - testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col5 from testgetfield;", new String [] {"5.5"}); - testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6", "select col6 from testgetfield;", new String [] {"F6"}); - testEval(schema, "testgetfield", "0,1,2,3,4.5,5.5,F6,abc,t", "select col8 from testgetfield;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col1 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col2 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col3 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col4 from table1;", new String [] {"4.5"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col5 from table1;", new String [] {"5.5"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col6 from table1;", new String [] {"F6"}); + testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6,abc,t", "select col8 from table1;", new String [] {"t"}); } @Test @@ -78,29 +78,29 @@ public void testNullHandling() throws IOException { schema.addColumn("col9", TajoDataTypes.Type.BOOLEAN); schema.addColumn("nullable", TajoDataTypes.Type.NULL_TYPE); - testEval(schema, "testnullhandling", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is null from testnullhandling;", new String[]{"t"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is null from testnullhandling;", new String [] {"t"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is null from testnullhandling;", new String [] {"t"}); - - testEval(schema, "testnullhandling", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is not null from testnullhandling;", new String [] {"f"}); - testEval(schema, "testnullhandling", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is not null from testnullhandling;", new String [] {"f"}); + testEval(schema, "table1", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is null from table1;", new String[]{"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is null from table1;", new String [] {"t"}); + + testEval(schema, "table1", ",1,2,3,4.5,6.5,F6,abc,abc,t", "select col0 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,,2,3,4.5,6.5,F6,abc,abc,t,", "select col1 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,,3,4.5,6.5,F6,abc,abc,t,", "select col2 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,,4.5,6.5,F6,abc,abc,t,", "select col3 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,,6.5,F6,abc,abc,t,", "select col4 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,,F6,abc,abc,t,", "select col5 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,,abc,abc,t,", "select col6 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,,abc,t,", "select col7 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,,t,", "select col8 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,,", "select col9 is not null from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5,F6,abc,abc,t,", "select nullable is not null from table1;", new String [] {"f"}); } @Test @@ -119,44 +119,44 @@ public void testComparison() throws IOException { testSimpleEval("select 1 < null;", new String [] {""}); testSimpleEval("select 1 <= null;", new String [] {""}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col1 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col2 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col3 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col4 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 = col5 from testcomparison;", new String [] {"f"}); - - testEval(inetSchema, "testcomparison", "192.168.0.1,192.168.0.1", "select addr1 = addr2 from testcomparison;", new String[]{"t"}); - testEval(inetSchema, "testcomparison", "192.168.0.1,192.168.0.2", "select addr1 = addr2 from testcomparison;", new String[]{"f"}); - - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col1 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col2 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col3 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col4 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <> col5 from testcomparison;", new String [] {"t"}); - - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col1 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col2 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col3 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col4 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 1 < col5 from testcomparison;", new String [] {"t"}); - - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col1 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col2 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col3 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col4 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 <= col5 from testcomparison;", new String [] {"t"}); - - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col1 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col2 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col3 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col4 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 > col5 from testcomparison;", new String [] {"f"}); - - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col1 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col2 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col3 from testcomparison;", new String [] {"t"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col4 from testcomparison;", new String [] {"f"}); - testEval(schema, "testcomparison", "0,1,2,3,4.5,6.5", "select 3 >= col5 from testcomparison;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 = col5 from table1;", new String [] {"f"}); + + testEval(inetSchema, "table1", "192.168.0.1,192.168.0.1", "select addr1 = addr2 from table1;", new String[]{"t"}); + testEval(inetSchema, "table1", "192.168.0.1,192.168.0.2", "select addr1 = addr2 from table1;", new String[]{"f"}); + + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <> col5 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1 < col5 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 <= col5 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 > col5 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 3 >= col5 from table1;", new String [] {"f"}); } @Test @@ -164,33 +164,33 @@ public void testBetweenAsymmetric() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TajoDataTypes.Type.INT4); schema.addColumn("col2", TajoDataTypes.Type.INT4); - testEval(schema, "testbetweenasymmetric", "0,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "1,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "2,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "3,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "4,", "select col1 between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "5,", "select (col2 between 1 and 3) is null from testbetweenasymmetric", new String[]{"t"}); - - testEval(schema, "testbetweenasymmetric", "0,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "1,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "2,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "3,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "4,", "select col1 between 3 and 1 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "5,", "select (col2 between 3 and 1) is null from testbetweenasymmetric", new String[]{"t"}); - - testEval(schema, "testbetweenasymmetric", "0,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "1,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "2,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "3,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"f"}); - testEval(schema, "testbetweenasymmetric", "4,", "select col1 not between 1 and 3 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "5,", "select (col2 not between 1 and 3) is null from testbetweenasymmetric", new String[]{"t"}); - - testEval(schema, "testbetweenasymmetric", "0,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "1,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "2,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "3,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "4,", "select col1 not between 3 and 1 from testbetweenasymmetric", new String[]{"t"}); - testEval(schema, "testbetweenasymmetric", "5,", "select (col2 not between 3 and 1) is null from testbetweenasymmetric", new String[]{"t"}); + testEval(schema, "table1", "0,", "select col1 between 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "1,", "select col1 between 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "2,", "select col1 between 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "3,", "select col1 between 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "4,", "select col1 between 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "5,", "select (col2 between 1 and 3) is null from table1", new String[]{"t"}); + + testEval(schema, "table1", "0,", "select col1 between 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "1,", "select col1 between 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "2,", "select col1 between 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "3,", "select col1 between 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "4,", "select col1 between 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "5,", "select (col2 between 3 and 1) is null from table1", new String[]{"t"}); + + testEval(schema, "table1", "0,", "select col1 not between 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "1,", "select col1 not between 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "2,", "select col1 not between 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "3,", "select col1 not between 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "4,", "select col1 not between 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "5,", "select (col2 not between 1 and 3) is null from table1", new String[]{"t"}); + + testEval(schema, "table1", "0,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "1,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "2,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "3,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "4,", "select col1 not between 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "5,", "select (col2 not between 3 and 1) is null from table1", new String[]{"t"}); } @Test @@ -198,33 +198,33 @@ public void testBetweenSymmetric() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TajoDataTypes.Type.INT4); schema.addColumn("col2", TajoDataTypes.Type.INT4); - testEval(schema, "testbetweensymmetric", "0,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "1,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "2,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "3,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "4,", "select col1 between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "5,", "select (col2 between symmetric 1 and 3) is null from testbetweensymmetric", new String[]{"t"}); - - testEval(schema, "testbetweensymmetric", "0,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "1,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "2,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "3,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "4,", "select col1 not between symmetric 1 and 3 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "5,", "select (col2 not between symmetric 1 and 3) is null from testbetweensymmetric", new String[]{"t"}); - - testEval(schema, "testbetweensymmetric", "0,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "1,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "2,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "3,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "4,", "select col1 between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "5,", "select (col2 between symmetric 3 and 1) is null from testbetweensymmetric", new String[]{"t"}); - - testEval(schema, "testbetweensymmetric", "0,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "1,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "2,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "3,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"f"}); - testEval(schema, "testbetweensymmetric", "4,", "select col1 not between symmetric 3 and 1 from testbetweensymmetric", new String[]{"t"}); - testEval(schema, "testbetweensymmetric", "5,", "select (col2 not between symmetric 3 and 1) is null from testbetweensymmetric", + testEval(schema, "table1", "0,", "select col1 between symmetric 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "1,", "select col1 between symmetric 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "2,", "select col1 between symmetric 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "3,", "select col1 between symmetric 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "4,", "select col1 between symmetric 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "5,", "select (col2 between symmetric 1 and 3) is null from table1", new String[]{"t"}); + + testEval(schema, "table1", "0,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "1,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "2,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "3,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"f"}); + testEval(schema, "table1", "4,", "select col1 not between symmetric 1 and 3 from table1", new String[]{"t"}); + testEval(schema, "table1", "5,", "select (col2 not between symmetric 1 and 3) is null from table1", new String[]{"t"}); + + testEval(schema, "table1", "0,", "select col1 between symmetric 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "1,", "select col1 between symmetric 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "2,", "select col1 between symmetric 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "3,", "select col1 between symmetric 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "4,", "select col1 between symmetric 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "5,", "select (col2 between symmetric 3 and 1) is null from table1", new String[]{"t"}); + + testEval(schema, "table1", "0,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "1,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "2,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "3,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"f"}); + testEval(schema, "table1", "4,", "select col1 not between symmetric 3 and 1 from table1", new String[]{"t"}); + testEval(schema, "table1", "5,", "select (col2 not between symmetric 3 and 1) is null from table1", new String[]{"t"}); } @@ -243,56 +243,56 @@ public void testUnary() throws IOException { // sign test - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from testunary;", new String [] {"1"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from testunary;", new String [] {"2"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from testunary;", new String [] {"3"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from testunary;", new String [] {"4.1"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from testunary;", new String [] {"5.1"}); - - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from testunary;", new String [] {"-1"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from testunary;", new String [] {"-2"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from testunary;", new String [] {"-3"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from testunary;", new String [] {"-4.1"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from testunary;", new String [] {"-5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from table1;", new String [] {"5.1"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from table1;", new String [] {"-1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from table1;", new String [] {"-2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from table1;", new String [] {"-3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from table1;", new String [] {"-4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from table1;", new String [] {"-5.1"}); // not test - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select col8 from testunary;", new String [] {"t"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select NOT (col8) from testunary;", new String [] {"f"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,t", "select NOT(NOT (col8)) from testunary;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select col8 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select NOT (col8) from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select NOT(NOT (col8)) from table1;", new String [] {"t"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,", "select col8 is null from testunary;", new String [] {"t"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,", "select (NOT (col8)) is null from testunary;", new String [] {"t"}); - testEval(schema, "testunary", "0,1,2,3,4.1,5.1,6,7,", "select (NOT(NOT (col8))) is null from testunary;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,", "select col8 is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,", "select (NOT (col8)) is null from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,", "select (NOT(NOT (col8))) is null from table1;", new String [] {"t"}); } @Test public void testAndOr() throws IOException { testSimpleEval("select true or (false or false) or false;", new String[] {"t"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true and true;", new String [] {"t"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true and false;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false and true;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false and false;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true and true;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true and false;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false and true;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false and false;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true or true;", new String [] {"t"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select true or false;", new String [] {"t"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false or true;", new String [] {"t"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select false or false;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true or true;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true or false;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false or true;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select false or false;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (true and true) and false;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (true and false) and true;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (false and true) and true;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (true and true) and false;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (true and false) and true;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (false and true) and true;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (1 < 2) and true;", new String [] {"t"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (1 < 2) and false;", new String [] {"f"}); - testEval(schema, "testandor", "0,1,2,3,4.5,6.5", "select (1 < 2) or false;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (1 < 2) and true;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (1 < 2) and false;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select (1 < 2) or false;", new String [] {"t"}); } @Test public void testFunction() throws IOException { - testEval(schema, "testfunction", "0,1,2,3,4.5,6.5", "select upper('abc');", new String [] {"ABC"}); - testEval(schema, "testfunction", "0,1,2,3,4.5,6.5", "select upper('bbc');", new String [] {"BBC"}); - testEval(schema, "testfunction", "0,1,2,3,4.5,6.5", "select upper('chs');", new String [] {"CHS"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('abc');", new String [] {"ABC"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('bbc');", new String [] {"BBC"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('chs');", new String [] {"CHS"}); testSimpleEval("select ltrim('xxtrim', 'xx') ", new String[]{"trim"}); } @@ -301,11 +301,11 @@ public void testFunction() throws IOException { public void testStringConcat() throws IOException { testSimpleEval("select length('123456') as col1 ", new String[]{"6"}); - testEval(schema, "teststringconcat", "0,1,2,3,4.5,6.5", "select 'abc' || 'bbc'", new String [] {"abcbbc"}); + testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 'abc' || 'bbc'", new String [] {"abcbbc"}); Schema schema = new Schema(); schema.addColumn("col1", TajoDataTypes.Type.TEXT); schema.addColumn("col2", TajoDataTypes.Type.TEXT); - testEval(schema, "teststringconcat", " trim, abc", "select ltrim(col1) || ltrim(col2) from teststringconcat", + testEval(schema, "table1", " trim, abc", "select ltrim(col1) || ltrim(col2) from table1", new String[]{"trimabc"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java index 59d441c1a7..94d5e712e8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java @@ -93,8 +93,8 @@ public void testNot() throws IOException { schema1.addColumn("col3", INT4); testEval(schema1, - "testnot", "123,123,456,-123", - "select col1 = col2, col1 = col3 from testnot", + "table1", "123,123,456,-123", + "select col1 = col2, col1 = col3 from table1", new String[]{"t", "f"}); } @@ -132,12 +132,12 @@ public void testComparisonEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "testcomparisonequal", "123,123,456,-123", - "select col1 = col2, col1 = col3, col1 = col4 from testcomparisonequal", + "table1", "123,123,456,-123", + "select col1 = col2, col1 = col3, col1 = col4 from table1", new String[]{"t", "f", "f"}); testEval(schema1, - "testcomparisonequal", "123,123,,", - "select col1 = col2, (col1 = col3) is null, (col3 = col2) is null from testcomparisonequal", + "table1", "123,123,,", + "select col1 = col2, (col1 = col3) is null, (col3 = col2) is null from table1", new String[]{"t", "t", "t"}); } @@ -149,12 +149,12 @@ public void testComparisonNotEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "testcomparisonnotequal", "123,123,456,-123", - "select col1 <> col2, col1 <> col3, col1 <> col4 from testcomparisonnotequal", + "table1", "123,123,456,-123", + "select col1 <> col2, col1 <> col3, col1 <> col4 from table1", new String[]{"f", "t", "t"}); testEval(schema1, - "testcomparisonnotequal", "123,123,,", - "select col1 <> col2, (col1 <> col3) is null, (col3 <> col2) is null from testcomparisonnotequal", + "table1", "123,123,,", + "select col1 <> col2, (col1 <> col3) is null, (col3 <> col2) is null from table1", new String[]{"f", "t", "t"}); } @@ -166,12 +166,12 @@ public void testComparisonLessThan() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "testcomparisonlessthan", "123,123,456,-123", - "select col1 < col2, col1 < col3, col1 < col4 from testcomparisonlessthan", + "table1", "123,123,456,-123", + "select col1 < col2, col1 < col3, col1 < col4 from table1", new String[]{"f", "t", "f"}); testEval(schema1, - "testcomparisonlessthan", "123,456,,", - "select col1 < col2, (col1 = col3) is null, (col4 = col1) is null from testcomparisonlessthan", + "table1", "123,456,,", + "select col1 < col2, (col1 = col3) is null, (col4 = col1) is null from table1", new String[]{"t", "t", "t"}); } @@ -183,12 +183,12 @@ public void testComparisonLessThanEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "testcomparisonlessthanequal", "123,123,456,-123", - "select col1 <= col2, col1 <= col3, col1 <= col4 from testcomparisonlessthanequal", + "table1", "123,123,456,-123", + "select col1 <= col2, col1 <= col3, col1 <= col4 from table1", new String[]{"t", "t", "f"}); testEval(schema1, - "testcomparisonlessthanequal", "123,456,,", - "select col1 <= col2, (col1 <= col3) is null, (col4 <= col1) is null from testcomparisonlessthanequal", + "table1", "123,456,,", + "select col1 <= col2, (col1 <= col3) is null, (col4 <= col1) is null from table1", new String[]{"t", "t", "t"}); } @@ -200,12 +200,12 @@ public void testComparisonGreaterThan() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "testcomparisongreaterthan", "123,123,456,-123", - "select col1 > col2, col3 > col2, col1 > col4 from testcomparisongreaterthan", + "table1", "123,123,456,-123", + "select col1 > col2, col3 > col2, col1 > col4 from table1", new String[]{"f", "t", "t"}); testEval(schema1, - "testcomparisongreaterthan", "123,456,,", - "select col2 > col1, col1 > col2, (col1 > col3) is null, (col4 > col1) is null from testcomparisongreaterthan", + "table1", "123,456,,", + "select col2 > col1, col1 > col2, (col1 > col3) is null, (col4 > col1) is null from table1", new String[]{"t", "f", "t", "t"}); } @@ -217,12 +217,12 @@ public void testComparisonGreaterThanEqual() throws IOException { schema1.addColumn("col3", INT4); schema1.addColumn("col4", INT4); testEval(schema1, - "testcomparisongreaterthanequal", "123,123,456,-123", - "select col1 >= col2, col3 >= col2, col1 >= col4 from testcomparisongreaterthanequal", + "table1", "123,123,456,-123", + "select col1 >= col2, col3 >= col2, col1 >= col4 from table1", new String[]{"t", "t", "t"}); testEval(schema1, - "testcomparisongreaterthanequal", "123,456,,", - "select col2 >= col1, col1 >= col2, (col1 >= col3) is null, (col4 >= col1) is null from testcomparisongreaterthanequal", + "table1", "123,456,,", + "select col2 >= col1, col1 >= col2, (col1 >= col3) is null, (col4 >= col1) is null from table1", new String[]{"t", "f", "t", "t"}); } @@ -238,20 +238,20 @@ public void testBetween() throws IOException { schema2.addColumn("col3", TEXT); // constant checker - testEval(schema2, "testbetween", "b,a,c", "select col1 between 'a' and 'c' from testbetween", new String[]{"t"}); - testEval(schema2, "testbetween", "b,a,c", "select col1 between 'c' and 'a' from testbetween", new String[]{"f"}); - testEval(schema2, "testbetween", "b,a,c", "select col1 between symmetric 'c' and 'a' from testbetween", new String[]{"t"}); - testEval(schema2, "testbetween", "d,a,c", "select col1 between 'a' and 'c' from testbetween", new String[]{"f"}); + testEval(schema2, "table1", "b,a,c", "select col1 between 'a' and 'c' from table1", new String[]{"t"}); + testEval(schema2, "table1", "b,a,c", "select col1 between 'c' and 'a' from table1", new String[]{"f"}); + testEval(schema2, "table1", "b,a,c", "select col1 between symmetric 'c' and 'a' from table1", new String[]{"t"}); + testEval(schema2, "table1", "d,a,c", "select col1 between 'a' and 'c' from table1", new String[]{"f"}); // tests for inclusive - testEval(schema2, "testbetween", "a,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"t"}); - testEval(schema2, "testbetween", "b,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"t"}); - testEval(schema2, "testbetween", "c,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"t"}); - testEval(schema2, "testbetween", "d,a,c", "select col1 between col2 and col3 from testbetween", new String[]{"f"}); + testEval(schema2, "table1", "a,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"}); + testEval(schema2, "table1", "b,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"}); + testEval(schema2, "table1", "c,a,c", "select col1 between col2 and col3 from table1", new String[]{"t"}); + testEval(schema2, "table1", "d,a,c", "select col1 between col2 and col3 from table1", new String[]{"f"}); // tests for asymmetric and symmetric - testEval(schema2, "testbetween", "b,a,c", "select col1 between col3 and col2 from testbetween", new String[]{"f"}); - testEval(schema2, "testbetween", "b,a,c", "select col1 between symmetric col3 and col2 from testbetween", new String[]{"t"}); + testEval(schema2, "table1", "b,a,c", "select col1 between col3 and col2 from table1", new String[]{"f"}); + testEval(schema2, "table1", "b,a,c", "select col1 between symmetric col3 and col2 from table1", new String[]{"t"}); } @Test @@ -266,27 +266,27 @@ public void testBetween2() throws IOException { // for TAJO-249 "case " + "when date_a BETWEEN 20130705 AND 20130715 AND ((date_b BETWEEN 20100101 AND 20120601) OR date_b > 20130715) " + "AND (date_c < 20120601 OR date_c > 20130715) AND date_d > 20130715" + - "then 1 else 0 end from testbetween2"; + "then 1 else 0 end from table1"; - testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "testbetween2", "20130716,20100102,20120525,20130716", query, new String [] {"0"}); + testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "table1", "20130716,20100102,20120525,20130716", query, new String [] {"0"}); // date_b - testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "testbetween2", "20130715,20120602,20120525,20130716", query, new String [] {"0"}); - testEval(schema3, "testbetween2", "20130715,20091201,20120525,20130716", query, new String [] {"0"}); - testEval(schema3, "testbetween2", "20130715,20130716,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "table1", "20130715,20120602,20120525,20130716", query, new String [] {"0"}); + testEval(schema3, "table1", "20130715,20091201,20120525,20130716", query, new String [] {"0"}); + testEval(schema3, "table1", "20130715,20130716,20120525,20130716", query, new String [] {"1"}); // date_c - testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "testbetween2", "20130715,20100102,20120602,20130716", query, new String [] {"0"}); + testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "table1", "20130715,20100102,20120602,20130716", query, new String [] {"0"}); - testEval(schema3, "testbetween2", "20130715,20100102,20130716,20130716", query, new String [] {"1"}); - testEval(schema3, "testbetween2", "20130715,20100102,20130714,20130716", query, new String [] {"0"}); + testEval(schema3, "table1", "20130715,20100102,20130716,20130716", query, new String [] {"1"}); + testEval(schema3, "table1", "20130715,20100102,20130714,20130716", query, new String [] {"0"}); // date_d - testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); - testEval(schema3, "testbetween2", "20130715,20100102,20120525,20130705", query, new String [] {"0"}); + testEval(schema3, "table1", "20130715,20100102,20120525,20130716", query, new String [] {"1"}); + testEval(schema3, "table1", "20130715,20100102,20120525,20130705", query, new String [] {"0"}); } ////////////////////////////////////////////////////////////////// @@ -300,21 +300,21 @@ public void testInPredicateWithConstant() throws IOException { schema2.addColumn("col2", TEXT); schema2.addColumn("col3", TEXT); - testEval(schema2, "testinpredicatewithconstant", "a,b,c", "select col1 in ('a'), col2 in ('a', 'c') from testinpredicatewithconstant", new String[]{"t","f"}); - testEval(schema2, "testinpredicatewithconstant", "a,,c", "select col1 in ('a','b','c'), (col2 in ('a', 'c')) is null from testinpredicatewithconstant", + testEval(schema2, "table1", "a,b,c", "select col1 in ('a'), col2 in ('a', 'c') from table1", new String[]{"t","f"}); + testEval(schema2, "table1", "a,,c", "select col1 in ('a','b','c'), (col2 in ('a', 'c')) is null from table1", new String[]{"t","t"}); testEval(schema2, - "testinpredicatewithconstant", + "table1", "2014-03-21,2015-04-01,2016-04-01", - "select substr(col1,1,4) in ('2014','2015','2016'), substr(col1,6,2)::int4 in (1,2,3) from testinpredicatewithconstant", + "select substr(col1,1,4) in ('2014','2015','2016'), substr(col1,6,2)::int4 in (1,2,3) from table1", new String[]{"t", "t"}); // null handling test testEval(schema2, - "testinpredicatewithconstant", + "table1", "2014-03-21,,2015-04-01", - "select (substr(col2,1,4)::int4 in (2014,2015,2016)) is null from testinpredicatewithconstant", + "select (substr(col2,1,4)::int4 in (2014,2015,2016)) is null from table1", new String[]{"t"}); } @@ -325,16 +325,16 @@ public void testInPredicateWithSimpleExprs() throws IOException { schema2.addColumn("col2", INT4); schema2.addColumn("col3", TEXT); - testEval(schema2, "testinpredicatewithsimpleexprs", "abc,2,3", "select col1 in ('a'||'b'||'c'), col2 in (1 + 1, 2 * 10, 2003) from testinpredicatewithsimpleexprs", + testEval(schema2, "table1", "abc,2,3", "select col1 in ('a'||'b'||'c'), col2 in (1 + 1, 2 * 10, 2003) from table1", new String[]{"t","t"}); - testEval(schema2, "testinpredicatewithsimpleexprs", "abc,2,3", "select col1 in ('a'||'b'), col2 in ('1'::int, '2'::int, 3) from testinpredicatewithsimpleexprs", + testEval(schema2, "table1", "abc,2,3", "select col1 in ('a'||'b'), col2 in ('1'::int, '2'::int, 3) from table1", new String[]{"f","t"}); testEval(schema2, - "testinpredicatewithsimpleexprs", + "table1", "abc,,3", - "select col1 in (reverse('cba')), (col2 in ('1'::int, '2'::int, 3)) is null from testinpredicatewithsimpleexprs", + "select col1 in (reverse('cba')), (col2 in ('1'::int, '2'::int, 3)) is null from table1", new String[]{"t","t"}); } @@ -347,9 +347,9 @@ public void testIsNullPredicate() throws IOException { Schema schema1 = new Schema(); schema1.addColumn("col1", INT4); schema1.addColumn("col2", INT4); - testEval(schema1, "testisnullpredicate", "123,", "select col1 is null, col2 is null as a from testisnullpredicate", + testEval(schema1, "table1", "123,", "select col1 is null, col2 is null as a from table1", new String[]{"f", "t"}); - testEval(schema1, "testisnullpredicate", "123,", "select col1 is not null, col2 is not null as a from testisnullpredicate", + testEval(schema1, "table1", "123,", "select col1 is not null, col2 is not null as a from table1", new String[]{"t", "f"}); } @@ -358,11 +358,11 @@ public void testIsNullPredicateWithFunction() throws IOException { Schema schema2 = new Schema(); schema2.addColumn("col1", TEXT); schema2.addColumn("col2", TEXT); - testEval(schema2, "testisnullpredicatewithfunction", "_123,", "select ltrim(col1, '_') is null, upper(col2) is null as a from testisnullpredicatewithfunction", + testEval(schema2, "table1", "_123,", "select ltrim(col1, '_') is null, upper(col2) is null as a from table1", new String[]{"f", "t"}); - testEval(schema2, "testisnullpredicatewithfunction", "_123,", - "select ltrim(col1, '_') is not null, upper(col2) is not null as a from testisnullpredicatewithfunction", new String[]{"t", "f"}); + testEval(schema2, "table1", "_123,", + "select ltrim(col1, '_') is not null, upper(col2) is not null as a from table1", new String[]{"t", "f"}); } ////////////////////////////////////////////////////////////////// @@ -397,10 +397,10 @@ public void testBooleanTestOnTable() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", BOOLEAN); schema.addColumn("col2", BOOLEAN); - testEval(schema, "testbooleantestontable", "t,f", "select col1 is true, col2 is false from testbooleantestontable", new String [] {"t", "t"}); - testEval(schema, "testbooleantestontable", "t,f", "select col1 is not true, col2 is not false from testbooleantestontable", + testEval(schema, "table1", "t,f", "select col1 is true, col2 is false from table1", new String [] {"t", "t"}); + testEval(schema, "table1", "t,f", "select col1 is not true, col2 is not false from table1", new String [] {"f", "f"}); - testEval(schema, "testbooleantestontable", "t,f", "select not col1 is not true, not col2 is not false from testbooleantestontable", + testEval(schema, "table1", "t,f", "select not col1 is not true, not col2 is not false from table1", new String [] {"t", "t"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java index 20a29c1184..b04d544862 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java @@ -106,69 +106,69 @@ public void testExplicitCast() throws IOException { testSimpleEval("select cast (1 as char)", new String[]{"1"}); testSimpleEval("select cast (119 as char)", new String[] {"1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int1 from testexplicitcast;", new String [] {"0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int1 from testexplicitcast;", new String [] {"1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int1 from testexplicitcast;", new String [] {"2"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int1 from testexplicitcast;", new String [] {"3"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int1 from testexplicitcast;", new String [] {"4"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int1 from testexplicitcast;", new String [] {"5"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int1 from testexplicitcast;", new String [] {"6"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int1 from testexplicitcast;", new String [] {"7"}); - - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int2 from testexplicitcast;", new String [] {"0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int2 from testexplicitcast;", new String [] {"1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int2 from testexplicitcast;", new String [] {"2"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int2 from testexplicitcast;", new String [] {"3"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int2 from testexplicitcast;", new String [] {"4"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int2 from testexplicitcast;", new String [] {"5"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int2 from testexplicitcast;", new String [] {"6"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int2 from testexplicitcast;", new String [] {"7"}); - - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int4 from testexplicitcast;", new String [] {"0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int4 from testexplicitcast;", new String [] {"1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int4 from testexplicitcast;", new String [] {"2"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int4 from testexplicitcast;", new String [] {"3"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int4 from testexplicitcast;", new String [] {"4"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int4 from testexplicitcast;", new String [] {"5"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int4 from testexplicitcast;", new String [] {"6"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int4 from testexplicitcast;", new String [] {"7"}); - - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::int8 from testexplicitcast;", new String [] {"0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::int8 from testexplicitcast;", new String [] {"1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::int8 from testexplicitcast;", new String [] {"2"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::int8 from testexplicitcast;", new String [] {"3"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::int8 from testexplicitcast;", new String [] {"4"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::int8 from testexplicitcast;", new String [] {"5"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::int8 from testexplicitcast;", new String [] {"6"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::int8 from testexplicitcast;", new String [] {"7"}); - - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::float4 from testexplicitcast;", new String [] {"0.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::float4 from testexplicitcast;", new String [] {"1.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::float4 from testexplicitcast;", new String [] {"2.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::float4 from testexplicitcast;", new String [] {"3.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::float4 from testexplicitcast;", new String [] {"4.1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::float4 from testexplicitcast;", new String [] {"5.1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::float4 from testexplicitcast;", new String [] {"6.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::float4 from testexplicitcast;", new String [] {"7.0"}); - - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::float8 from testexplicitcast;", new String [] {"0.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::float8 from testexplicitcast;", new String [] {"1.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::float8 from testexplicitcast;", new String [] {"2.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::float8 from testexplicitcast;", new String [] {"3.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::float8 from testexplicitcast;", new String[] + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int1 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int1 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int1 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int1 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int1 from table1;", new String [] {"5"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int1 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int1 from table1;", new String [] {"7"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int2 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int2 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int2 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int2 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int2 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int2 from table1;", new String [] {"5"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int2 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int2 from table1;", new String [] {"7"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int4 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int4 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int4 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int4 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int4 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int4 from table1;", new String [] {"5"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int4 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int4 from table1;", new String [] {"7"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::int8 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::int8 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::int8 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::int8 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::int8 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::int8 from table1;", new String [] {"5"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::int8 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::int8 from table1;", new String [] {"7"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::float4 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::float4 from table1;", new String [] {"1.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::float4 from table1;", new String [] {"2.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::float4 from table1;", new String [] {"3.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::float4 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::float4 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::float4 from table1;", new String [] {"6.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::float4 from table1;", new String [] {"7.0"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::float8 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::float8 from table1;", new String [] {"1.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::float8 from table1;", new String [] {"2.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::float8 from table1;", new String [] {"3.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::float8 from table1;", new String[] {Double.valueOf(4.1f).toString()}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::float8 from testexplicitcast;", new String [] {"5.1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::float8 from testexplicitcast;", new String [] {"6.0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::float8 from testexplicitcast;", new String [] {"7.0"}); - - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col0::text from testexplicitcast;", new String [] {"0"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col1::text from testexplicitcast;", new String [] {"1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col2::text from testexplicitcast;", new String [] {"2"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col3::text from testexplicitcast;", new String [] {"3"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col4::text from testexplicitcast;", new String [] {"4.1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col5::text from testexplicitcast;", new String [] {"5.1"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col6::text from testexplicitcast;", new String [] {"6"}); - testEval(schema, "testexplicitcast", "0,1,2,3,4.1,5.1,6,7", "select col7::text from testexplicitcast;", new String [] {"7"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::float8 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::float8 from table1;", new String [] {"6.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::float8 from table1;", new String [] {"7.0"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0::text from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1::text from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2::text from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3::text from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4::text from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5::text from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col6::text from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col7::text from table1;", new String [] {"7"}); } @Test @@ -183,94 +183,94 @@ public void testImplicitCastForInt1() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col0 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col1 from testimplicitcastforint1;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col2 from testimplicitcastforint1;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col3 from testimplicitcastforint1;", new String [] {"3"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col4 from testimplicitcastforint1;", new String [] {"4.1"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col5 from testimplicitcastforint1;", new String [] {"5.1"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col6::int1 from testimplicitcastforint1;", new String [] {"6"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col7::int1 from testimplicitcastforint1;", new String [] {"7"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col0 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col1 from testimplicitcastforint1;", new String [] {"-1"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col2 from testimplicitcastforint1;", new String [] {"-2"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col3 from testimplicitcastforint1;", new String [] {"-3"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col4 from testimplicitcastforint1;", new String [] {"-4.1"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col5 from testimplicitcastforint1;", new String [] {"-5.1"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col6::int1 from testimplicitcastforint1;", new String [] {"-6"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col7::int1 from testimplicitcastforint1;", new String [] {"-7"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col0 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col1 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col2 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col3 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col4 from testimplicitcastforint1;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col5 from testimplicitcastforint1;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col6::int1 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col7::int1 from testimplicitcastforint1;", new String [] {"0"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col1 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col2 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col3 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col4 from testimplicitcastforint1;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col5 from testimplicitcastforint1;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col6::int1 from testimplicitcastforint1;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col7::int1 from testimplicitcastforint1;", new String [] {"0"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col0 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col1 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col2 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col3 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col4 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col5 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col6::int1 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col7::int1 from testimplicitcastforint1;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col0 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col1 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col2 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col3 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col4 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col5 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col6::int1 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col7::int1 from testimplicitcastforint1;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col0 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col1 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col2 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col3 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col4 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col5 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col6::int1 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col7::int1 from testimplicitcastforint1;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col0 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col1 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col2 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col3 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col4 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col5 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col6::int1 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col7::int1 from testimplicitcastforint1;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col0 from testimplicitcastforint1;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col1 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col2 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col3 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col4 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col5 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col6::int1 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col7::int1 from testimplicitcastforint1;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col0 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col1 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col2 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col3 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col4 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col5 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col6::int1 from testimplicitcastforint1;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col7::int1 from testimplicitcastforint1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col0 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col1 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col2 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col3 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col4 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col5 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col6::int1 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 + col7::int1 from table1;", new String [] {"7"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col0 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col1 from table1;", new String [] {"-1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col2 from table1;", new String [] {"-2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col3 from table1;", new String [] {"-3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col4 from table1;", new String [] {"-4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col5 from table1;", new String [] {"-5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col6::int1 from table1;", new String [] {"-6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 - col7::int1 from table1;", new String [] {"-7"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col0 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col2 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col3 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col4 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col5 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col6::int1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 * col7::int1 from table1;", new String [] {"0"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col2 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col3 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col4 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col5 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col6::int1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 % col7::int1 from table1;", new String [] {"0"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col6::int1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 = col7::int1 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col6::int1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <> col7::int1 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col6::int1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 > col7::int1 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col6::int1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 >= col7::int1 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col6::int1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 < col7::int1 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col6::int1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col0 <= col7::int1 from table1;", new String [] {"t"}); } @Test @@ -285,94 +285,94 @@ public void testImplicitCastForInt2() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col0 from testimplicitcastforint2;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col1 from testimplicitcastforint2;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col2 from testimplicitcastforint2;", new String [] {"3"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col3 from testimplicitcastforint2;", new String [] {"4"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col4 from testimplicitcastforint2;", new String [] {"5.1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col5 from testimplicitcastforint2;", new String [] {"6.1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col6::int2 from testimplicitcastforint2;", new String [] {"7"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 + col7::int2 from testimplicitcastforint2;", new String [] {"8"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col0 from testimplicitcastforint2;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col1 from testimplicitcastforint2;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col2 from testimplicitcastforint2;", new String [] {"-1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col3 from testimplicitcastforint2;", new String [] {"-2"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col4 from testimplicitcastforint2;", new String [] {"-3.1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col5 from testimplicitcastforint2;", new String [] {"-4.1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col6::int2 from testimplicitcastforint2;", new String [] {"-5"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 - col7::int2 from testimplicitcastforint2;", new String [] {"-6"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col0 from testimplicitcastforint2;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col1 from testimplicitcastforint2;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col2 from testimplicitcastforint2;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col3 from testimplicitcastforint2;", new String [] {"3"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col4 from testimplicitcastforint2;", new String [] {"4.1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col5 from testimplicitcastforint2;", new String [] {"5.1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col6::int2 from testimplicitcastforint2;", new String [] {"6"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 * col7::int2 from testimplicitcastforint2;", new String [] {"7"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col1 from testimplicitcastforint2;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col2 from testimplicitcastforint2;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col3 from testimplicitcastforint2;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col4 from testimplicitcastforint2;", new String [] {"1.0"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col5 from testimplicitcastforint2;", new String [] {"1.0"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col6::int2 from testimplicitcastforint2;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 % col7::int2 from testimplicitcastforint2;", new String [] {"1"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col0 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col1 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col2 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col3 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col4 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col5 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col6::int2 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 = col7::int2 from testimplicitcastforint2;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col0 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col1 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col2 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col3 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col4 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col5 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col6::int2 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col7::int2 from testimplicitcastforint2;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col0 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col1 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col2 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col3 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col4 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col5 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col6::int2 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 > col7::int2 from testimplicitcastforint2;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col0 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col1 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col2 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col3 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col4 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col5 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col6::int2 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col7::int2 from testimplicitcastforint2;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col0 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col1 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col2 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col3 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col4 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col5 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col6::int2 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 < col7::int2 from testimplicitcastforint2;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col0 from testimplicitcastforint2;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col1 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col2 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col3 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col4 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col5 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col6::int2 from testimplicitcastforint2;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint2", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col7::int2 from testimplicitcastforint2;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col0 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col1 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col2 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col3 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col4 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col5 from table1;", new String [] {"6.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col6::int2 from table1;", new String [] {"7"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 + col7::int2 from table1;", new String [] {"8"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col0 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col2 from table1;", new String [] {"-1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col3 from table1;", new String [] {"-2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col4 from table1;", new String [] {"-3.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col5 from table1;", new String [] {"-4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col6::int2 from table1;", new String [] {"-5"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 - col7::int2 from table1;", new String [] {"-6"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col0 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col1 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col2 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col3 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col4 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col5 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col6::int2 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 * col7::int2 from table1;", new String [] {"7"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col2 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col3 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col4 from table1;", new String [] {"1.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col5 from table1;", new String [] {"1.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col6::int2 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 % col7::int2 from table1;", new String [] {"1"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col6::int2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 = col7::int2 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col6::int2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <> col7::int2 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col6::int2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 > col7::int2 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col6::int2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 >= col7::int2 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col6::int2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 < col7::int2 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col6::int2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col1 <= col7::int2 from table1;", new String [] {"t"}); } @Test @@ -387,95 +387,95 @@ public void testImplicitCastForInt4() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col0 from testimplicitcastforint4;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col1 from testimplicitcastforint4;", new String [] {"3"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col2 from testimplicitcastforint4;", new String [] {"4"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col3 from testimplicitcastforint4;", new String [] {"5"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col4 from testimplicitcastforint4;", new String [] {"6.1"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col5 from testimplicitcastforint4;", new String [] {"7.1"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col6::int4 from testimplicitcastforint4;", new String [] {"8"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 + col7::int4 from testimplicitcastforint4;", new String [] {"9"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col0 from testimplicitcastforint4;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col1 from testimplicitcastforint4;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col2 from testimplicitcastforint4;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col3 from testimplicitcastforint4;", new String [] {"-1"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col4 from testimplicitcastforint4;", new String [] {"-2.1"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col5 from testimplicitcastforint4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col0 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col1 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col2 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col3 from table1;", new String [] {"5"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col4 from table1;", new String [] {"6.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col5 from table1;", new String [] {"7.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col6::int4 from table1;", new String [] {"8"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 + col7::int4 from table1;", new String [] {"9"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col0 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col1 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col2 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col3 from table1;", new String [] {"-1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col4 from table1;", new String [] {"-2.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col5 from table1;", new String [] { (new Integer(2) - 5.1d) +""}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col6::int4 from testimplicitcastforint4;", new String [] {"-4"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 - col7::int4 from testimplicitcastforint4;", new String [] {"-5"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col0 from testimplicitcastforint4;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col1 from testimplicitcastforint4;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col2 from testimplicitcastforint4;", new String [] {"4"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col3 from testimplicitcastforint4;", new String [] {"6"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col4 from testimplicitcastforint4;", new String [] {"8.2"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col5 from testimplicitcastforint4;", new String [] {"10.2"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col6::int4 from testimplicitcastforint4;", new String [] {"12"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 * col7::int4 from testimplicitcastforint4;", new String [] {"14"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col1 from testimplicitcastforint4;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col2 from testimplicitcastforint4;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col3 from testimplicitcastforint4;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col4 from testimplicitcastforint4;", new String [] {"2.0"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col5 from testimplicitcastforint4;", new String [] {"2.0"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col6::int4 from testimplicitcastforint4;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 % col7::int4 from testimplicitcastforint4;", new String [] {"2"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col0 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col1 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col2 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col3 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col4 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col5 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col6::int4 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 = col7::int4 from testimplicitcastforint4;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col0 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col1 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col2 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col3 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col4 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col5 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col6::int4 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col7::int4 from testimplicitcastforint4;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col0 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col1 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col2 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col3 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col4 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col5 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col6::int4 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 > col7::int4 from testimplicitcastforint4;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col0 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col1 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col2 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col3 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col4 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col5 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col6::int4 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col7::int4 from testimplicitcastforint4;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col0 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col1 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col2 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col3 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col4 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col5 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col6::int4 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 < col7::int4 from testimplicitcastforint4;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col0 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col1 from testimplicitcastforint4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col2 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col3 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col4 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col5 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col6::int4 from testimplicitcastforint4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint4", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col7::int4 from testimplicitcastforint4;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col6::int4 from table1;", new String [] {"-4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 - col7::int4 from table1;", new String [] {"-5"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col0 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col1 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col2 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col3 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col4 from table1;", new String [] {"8.2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col5 from table1;", new String [] {"10.2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col6::int4 from table1;", new String [] {"12"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 * col7::int4 from table1;", new String [] {"14"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col2 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col3 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col4 from table1;", new String [] {"2.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col5 from table1;", new String [] {"2.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col6::int4 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 % col7::int4 from table1;", new String [] {"2"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col6::int4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 = col7::int4 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col6::int4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <> col7::int4 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col6::int4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 > col7::int4 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col6::int4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 >= col7::int4 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col6::int4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 < col7::int4 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col6::int4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col2 <= col7::int4 from table1;", new String [] {"t"}); } @Test @@ -490,99 +490,99 @@ public void testImplicitCastForInt8() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col0 from testimplicitcastforint8;", new String[]{"3"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col1 from testimplicitcastforint8;", new String [] {"4"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col2 from testimplicitcastforint8;", new String [] {"5"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col3 from testimplicitcastforint8;", new String [] {"6"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col4 from testimplicitcastforint8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col0 from table1;", new String[]{"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col1 from table1;", new String [] {"4"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col2 from table1;", new String [] {"5"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col3 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col4 from table1;", new String [] { (new Long(3) + new Float(4.1))+""}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col5 from testimplicitcastforint8;", new String [] {"8.1"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col6::int8 from testimplicitcastforint8;", new String [] {"9"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 + col7::int8 from testimplicitcastforint8;", new String [] {"10"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col0 from testimplicitcastforint8;", new String [] {"3"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col1 from testimplicitcastforint8;", new String [] {"2"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col2 from testimplicitcastforint8;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col3 from testimplicitcastforint8;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col4 from testimplicitcastforint8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col5 from table1;", new String [] {"8.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col6::int8 from table1;", new String [] {"9"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 + col7::int8 from table1;", new String [] {"10"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col0 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col1 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col2 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col3 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col4 from table1;", new String [] { (new Long(3) - new Float(4.1))+""}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col5 from testimplicitcastforint8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col5 from table1;", new String [] { (new Long(3) - 5.1d)+""}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col6::int8 from testimplicitcastforint8;", new String [] {"-3"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 - col7::int8 from testimplicitcastforint8;", new String [] {"-4"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col0 from testimplicitcastforint8;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col1 from testimplicitcastforint8;", new String [] {"3"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col2 from testimplicitcastforint8;", new String [] {"6"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col3 from testimplicitcastforint8;", new String [] {"9"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col4 from testimplicitcastforint8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col6::int8 from table1;", new String [] {"-3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 - col7::int8 from table1;", new String [] {"-4"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col0 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col1 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col2 from table1;", new String [] {"6"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col3 from table1;", new String [] {"9"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col4 from table1;", new String [] { (new Long(3) * new Float("4.1"))+""}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col5 from testimplicitcastforint8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col5 from table1;", new String [] { (new Long(3) * new Double("5.1"))+""}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col6::int8 from testimplicitcastforint8;", new String [] {"18"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 * col7::int8 from testimplicitcastforint8;", new String [] {"21"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col1 from testimplicitcastforint8;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col2 from testimplicitcastforint8;", new String [] {"1"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col3 from testimplicitcastforint8;", new String [] {"0"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col4 from testimplicitcastforint8;", new String [] {"3.0"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col5 from testimplicitcastforint8;", new String [] {"3.0"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col6::int8 from testimplicitcastforint8;", new String [] {"3"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 % col7::int8 from testimplicitcastforint8;", new String [] {"3"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col0 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col1 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col2 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col3 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col4 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col5 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col6::int8 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 = col7::int8 from testimplicitcastforint8;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col0 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col1 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col2 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col3 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col4 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col5 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col6::int8 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col7::int8 from testimplicitcastforint8;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col0 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col1 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col2 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col3 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col4 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col5 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col6::int8 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 > col7::int8 from testimplicitcastforint8;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col0 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col1 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col2 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col3 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col4 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col5 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col6::int8 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col7::int8 from testimplicitcastforint8;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col0 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col1 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col2 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col3 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col4 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col5 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col6::int8 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 < col7::int8 from testimplicitcastforint8;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col0 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col1 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col2 from testimplicitcastforint8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col3 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col4 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col5 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col6::int8 from testimplicitcastforint8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforint8", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col7::int8 from testimplicitcastforint8;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col6::int8 from table1;", new String [] {"18"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 * col7::int8 from table1;", new String [] {"21"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col1 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col2 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col3 from table1;", new String [] {"0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col4 from table1;", new String [] {"3.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col5 from table1;", new String [] {"3.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col6::int8 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 % col7::int8 from table1;", new String [] {"3"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col6::int8 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 = col7::int8 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col6::int8 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <> col7::int8 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col6::int8 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 > col7::int8 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col6::int8 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 >= col7::int8 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col6::int8 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 < col7::int8 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col6::int8 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col3 <= col7::int8 from table1;", new String [] {"t"}); } @Test @@ -597,111 +597,111 @@ public void testImplicitCastForFloat4() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col0 from testimplicitcastforfloat4;", new String [] {"4.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col1 from testimplicitcastforfloat4;", new String [] {"5.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col2 from testimplicitcastforfloat4;", new String [] {"6.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col3 from testimplicitcastforfloat4;", new String [] {"7.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col4 from testimplicitcastforfloat4;", new String [] {"8.2"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col5 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col0 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col1 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col2 from table1;", new String [] {"6.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col3 from table1;", new String [] {"7.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col4 from table1;", new String [] {"8.2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col5 from table1;", new String [] { (new Float(4.1) + 5.1d)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col6::float4 from testimplicitcastforfloat4;", new String [] {"10.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 + col7::float4 from testimplicitcastforfloat4;", new String [] {"11.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col6::float4 from table1;", new String [] {"10.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 + col7::float4 from table1;", new String [] {"11.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col0 from testimplicitcastforfloat4;", new String [] {"4.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col1 from testimplicitcastforfloat4;", new String [] {"3.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col2 from testimplicitcastforfloat4;", new String [] {"2.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col3 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col0 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col1 from table1;", new String [] {"3.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col2 from table1;", new String [] {"2.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col3 from table1;", new String [] { (new Float(4.1) - new Long(3))+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col4 from testimplicitcastforfloat4;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col5 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col4 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col5 from table1;", new String [] { (new Float(4.1) - 5.1d)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col6::float4 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col6::float4 from table1;", new String [] { (4.1f - 6f)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 - col7::float4 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 - col7::float4 from table1;", new String [] { (4.1f - 7f)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col0 from testimplicitcastforfloat4;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col1 from testimplicitcastforfloat4;", new String [] {"4.1"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col2 from testimplicitcastforfloat4;", new String [] {"8.2"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col3 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col0 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col1 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col2 from table1;", new String [] {"8.2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col3 from table1;", new String [] { (new Float(4.1) * new Long(3))+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col4 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col4 from table1;", new String [] { (new Float(4.1) * new Float(4.1))+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col5 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col5 from table1;", new String [] { (new Float(4.1) * 5.1d)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col6::float4 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col6::float4 from table1;", new String [] { (new Float(4.1) * 6f)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 * col7::float4 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 * col7::float4 from table1;", new String [] { (new Float(4.1) * 7f)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col1 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col1 from table1;", new String [] { (new Float(4.1) % new Integer(1))+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col2 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col2 from table1;", new String [] { (new Float(4.1) % new Integer(2))+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col3 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col3 from table1;", new String [] { (new Float(4.1) % new Long(3))+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col4 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col4 from table1;", new String [] { (new Float(4.1) % new Float(4.1))+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col5 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col5 from table1;", new String [] { (new Float(4.1) % 5.1d)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col6::float4 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col6::float4 from table1;", new String [] { (new Float(4.1) % 6f)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 % col7::int1 from testimplicitcastforfloat4;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 % col7::int1 from table1;", new String [] { (new Float(4.1) % 7f)+""}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col0 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col1 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col2 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col3 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col4 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col5 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col6::int1 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 = col7::int1 from testimplicitcastforfloat4;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col0 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col1 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col2 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col3 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col4 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col5 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col6::int1 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col7::int1 from testimplicitcastforfloat4;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col0 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col1 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col2 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col3 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col4 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col5 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col6::int1 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 > col7::int1 from testimplicitcastforfloat4;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col0 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col1 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col2 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col3 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col4 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col5 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col6::int1 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col7::int1 from testimplicitcastforfloat4;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col0 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col1 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col2 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col3 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col4 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col5 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col6::int1 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 < col7::int1 from testimplicitcastforfloat4;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col0 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col1 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col2 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col3 from testimplicitcastforfloat4;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col4 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col5 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col6::int1 from testimplicitcastforfloat4;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat4", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col7::int1 from testimplicitcastforfloat4;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col6::int1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 = col7::int1 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col6::int1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <> col7::int1 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col6::int1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 > col7::int1 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col6::int1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 >= col7::int1 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col6::int1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 < col7::int1 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col6::int1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col4 <= col7::int1 from table1;", new String [] {"t"}); } @Test @@ -716,112 +716,112 @@ public void testImplicitCastForFloat8() throws IOException { schema.addColumn("col6", TajoDataTypes.Type.TEXT); schema.addColumn("col7", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col0 from testimplicitcastforfloat8;", new String [] {"5.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col1 from testimplicitcastforfloat8;", new String [] {"6.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col2 from testimplicitcastforfloat8;", new String [] {"7.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col3 from testimplicitcastforfloat8;", new String [] {"8.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col4 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col0 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col1 from table1;", new String [] {"6.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col2 from table1;", new String [] {"7.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col3 from table1;", new String [] {"8.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col4 from table1;", new String [] { (5.1d + 4.1f)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col5 from testimplicitcastforfloat8;", new String [] {"10.2"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col6::int1 from testimplicitcastforfloat8;", new String [] {"11.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 + col7::int1 from testimplicitcastforfloat8;", new String [] {"12.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col5 from table1;", new String [] {"10.2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col6::int1 from table1;", new String [] {"11.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 + col7::int1 from table1;", new String [] {"12.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col0 from testimplicitcastforfloat8;", new String [] {"5.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col1 from testimplicitcastforfloat8;", new String [] {"4.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col2 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col0 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col1 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col2 from table1;", new String [] { (5.1d - new Integer(2))+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col3 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col3 from table1;", new String [] { (5.1d - 3l)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col4 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col4 from table1;", new String [] { (5.1d - 4.1f)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col5 from testimplicitcastforfloat8;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col6::float8 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col5 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col6::float8 from table1;", new String [] { (5.1d - 6d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 - col7::float8 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 - col7::float8 from table1;", new String [] { (5.1d - 7d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col0 from testimplicitcastforfloat8;", new String [] {"0.0"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col1 from testimplicitcastforfloat8;", new String [] {"5.1"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col2 from testimplicitcastforfloat8;", new String [] {"10.2"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col3 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col0 from table1;", new String [] {"0.0"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col1 from table1;", new String [] {"5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col2 from table1;", new String [] {"10.2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col3 from table1;", new String [] { (5.1d * new Long(3))+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col4 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col4 from table1;", new String [] { (5.1d * new Float(4.1))+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col5 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col5 from table1;", new String [] { (5.1d * 5.1d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col6::float8 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col6::float8 from table1;", new String [] { (5.1d * 6d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 * col7::float8 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 * col7::float8 from table1;", new String [] { (5.1d * 7d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col1 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col1 from table1;", new String [] { (5.1d % new Integer(1))+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col2 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col2 from table1;", new String [] { (5.1d % new Integer(2))+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col3 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col3 from table1;", new String [] { (5.1d % new Long(3))+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col4 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col4 from table1;", new String [] { (5.1d % new Float(4.1))+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col5 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col5 from table1;", new String [] { (5.1d % 5.1d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col6::float8 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col6::float8 from table1;", new String [] { (5.1d % 6d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 % col7::float8 from testimplicitcastforfloat8;", new String [] { + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 % col7::float8 from table1;", new String [] { (5.1d % 7d)+""}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col0 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col1 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col2 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col3 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col4 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col5 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col6::float8 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 = col7::float8 from testimplicitcastforfloat8;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col0 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col1 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col2 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col3 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col4 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col5 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col6::float8 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col7::float8 from testimplicitcastforfloat8;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col0 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col1 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col2 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col3 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col4 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col5 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col6::float8 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 > col7::float8 from testimplicitcastforfloat8;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col0 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col1 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col2 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col3 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col4 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col5 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col6::float8 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col7::float8 from testimplicitcastforfloat8;", new String [] {"f"}); - - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col0 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col1 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col2 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col3 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col4 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col5 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col6::float8 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 < col7::float8 from testimplicitcastforfloat8;", new String [] {"t"}); - - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col0 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col1 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col2 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col3 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col4 from testimplicitcastforfloat8;", new String [] {"f"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col5 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col6::float8 from testimplicitcastforfloat8;", new String [] {"t"}); - testEval(schema, "testimplicitcastforfloat8", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col7::float8 from testimplicitcastforfloat8;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col6::float8 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 = col7::float8 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col6::float8 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <> col7::float8 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col6::float8 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 > col7::float8 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col0 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col1 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col2 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col3 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col4 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col6::float8 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 >= col7::float8 from table1;", new String [] {"f"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col5 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col6::float8 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 < col7::float8 from table1;", new String [] {"t"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col0 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col1 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col2 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col3 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col4 from table1;", new String [] {"f"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col5 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col6::float8 from table1;", new String [] {"t"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7", "select col5 <= col7::float8 from table1;", new String [] {"t"}); } @Test @@ -839,17 +839,17 @@ public void testSigned() throws IOException { // sign test - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from testsigned;", new String [] {"1"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from testsigned;", new String [] {"2"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from testsigned;", new String [] {"3"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from testsigned;", new String [] {"4.1"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from testsigned;", new String [] {"5.1"}); - - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from testsigned;", new String [] {"-1"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from testsigned;", new String [] {"-2"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from testsigned;", new String [] {"-3"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from testsigned;", new String [] {"-4.1"}); - testEval(schema, "testsigned", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from testsigned;", new String [] {"-5.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col1 from table1;", new String [] {"1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col2 from table1;", new String [] {"2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col3 from table1;", new String [] {"3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col4 from table1;", new String [] {"4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select +col5 from table1;", new String [] {"5.1"}); + + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col1 from table1;", new String [] {"-1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col2 from table1;", new String [] {"-2"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col3 from table1;", new String [] {"-3"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col4 from table1;", new String [] {"-4.1"}); + testEval(schema, "table1", "0,1,2,3,4.1,5.1,6,7,t", "select -col5 from table1;", new String [] {"-5.1"}); } @Test @@ -875,18 +875,18 @@ public void testCastFromTable() throws IOException { schema.addColumn("col2", TEXT); testEval(queryContext, schema, - "testcastfromtable", + "table1", "123,234", - "select cast(col1 as float) as b, cast(col2 as float) as a from testcastfromtable", + "select cast(col1 as float) as b, cast(col2 as float) as a from table1", new String[]{"123.0", "234.0"}); - testEval(queryContext, schema, "testcastfromtable", "123,234", "select col1::float, col2::float from testcastfromtable", + testEval(queryContext, schema, "table1", "123,234", "select col1::float, col2::float from table1", new String[]{"123.0", "234.0"}); TimestampDatum timestamp = DatumFactory.createTimestamp("1980-04-01 01:50:01" + DateTimeUtil.getTimeZoneDisplayTime(tz)); - testEval(queryContext, schema, "testcastfromtable", "1980-04-01 01:50:01,234", - "select col1::timestamp as t1, col2::float from testcastfromtable where t1 = '1980-04-01 01:50:01'::timestamp", + testEval(queryContext, schema, "table1", "1980-04-01 01:50:01,234", + "select col1::timestamp as t1, col2::float from table1 where t1 = '1980-04-01 01:50:01'::timestamp", new String[]{TimestampDatum.asChars(timestamp.asTimeMeta(), tz, false), "234.0"} ); @@ -905,7 +905,7 @@ public void testBooleanLiteral() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); - testEval(schema, "testbooleanliteral", "123,234", "select col1, col2 from testbooleanliteral where true", new String[]{"123", "234"}); + testEval(schema, "table1", "123,234", "select col1, col2 from table1 where true", new String[]{"123", "234"}); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java index c556b5cf92..705aa69eb1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java @@ -58,10 +58,10 @@ public void testMaxLongWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("value", TajoDataTypes.Type.INT8); String[] data = new String[]{ "1|-111", "2|\\N", "3|-333" }; - TajoTestingCluster.createTable("testmaxlongwithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select max(value) as max_value from testmaxlongwithnull"); + ResultSet res = executeString("select max(value) as max_value from testbuiltin11"); String ascExpected = "max_value\n" + "-------------------------------\n" + "-111\n"; @@ -69,7 +69,7 @@ public void testMaxLongWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testmaxlongwithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -83,10 +83,10 @@ public void testMinMaxDate() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.DATE); String[] data = new String[]{ "2014-01-02", "2014-12-01", "2015-01-01", "1999-08-09", "2000-03-01" }; - TajoTestingCluster.createTable("testminmaxdate", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxdate"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "1999-08-09,2015-01-01\n"; @@ -94,7 +94,7 @@ public void testMinMaxDate() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminmaxdate PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -107,10 +107,10 @@ public void testMinMaxDateWithNull() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.DATE); String[] data = new String[]{ "2014-01-02", "2014-12-01", "\\N", "\\N", "2000-03-01" }; - TajoTestingCluster.createTable("testminmaxdatewithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxdatewithnull"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "2000-03-01,2014-12-01\n"; @@ -118,7 +118,7 @@ public void testMinMaxDateWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminmaxdatewithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -131,10 +131,10 @@ public void testMinMaxTime() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.TIME); String[] data = new String[]{ "11:11:11", "23:12:50", "00:00:01", "09:59:59", "12:13:14" }; - TajoTestingCluster.createTable("testminmaxtime", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtime"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "00:00:01,23:12:50\n"; @@ -142,7 +142,7 @@ public void testMinMaxTime() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminmaxtime PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -155,10 +155,10 @@ public void testMinMaxTimeWithNull() throws Exception { Schema schema = new Schema(); schema.addColumn("value", TajoDataTypes.Type.TIME); String[] data = new String[]{ "11:11:11", "\\N", "\\N", "09:59:59", "12:13:14" }; - TajoTestingCluster.createTable("testminmaxtimewithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtimewithnull"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "09:59:59,12:13:14\n"; @@ -166,7 +166,7 @@ public void testMinMaxTimeWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminmaxtimewithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -180,10 +180,10 @@ public void testMinMaxTimestamp() throws Exception { schema.addColumn("value", TajoDataTypes.Type.TIMESTAMP); String[] data = new String[]{ "1999-01-01 11:11:11", "2015-01-01 23:12:50", "2016-12-24 00:00:01", "1977-05-04 09:59:59", "2002-11-21 12:13:14" }; - TajoTestingCluster.createTable("testminmaxtimestamp", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtimestamp"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "1977-05-04 09:59:59,2016-12-24 00:00:01\n"; @@ -191,7 +191,7 @@ public void testMinMaxTimestamp() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminmaxtimestamp PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -205,10 +205,10 @@ public void testMinMaxTimestampWithNull() throws Exception { schema.addColumn("value", TajoDataTypes.Type.TIMESTAMP); String[] data = new String[]{ "1999-01-01 11:11:11", "2015-01-01 23:12:50", "\\N", "\\N", "2002-11-21 12:13:14" }; - TajoTestingCluster.createTable("testminmaxtimestampwithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testminmaxtimestampwithnull"); + ResultSet res = executeString("select min(value) as min_value, max(value) as max_value from testbuiltin11"); String ascExpected = "min_value,max_value\n" + "-------------------------------\n" + "1999-01-01 11:11:11,2015-01-01 23:12:50\n"; @@ -216,7 +216,7 @@ public void testMinMaxTimestampWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminmaxtimestampwithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -237,10 +237,10 @@ public void testMinLongWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("value", TajoDataTypes.Type.INT8); String[] data = new String[]{ "1|111", "2|\\N", "3|333" }; - TajoTestingCluster.createTable("testminlongwithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(value) as min_value from testminlongwithnull"); + ResultSet res = executeString("select min(value) as min_value from testbuiltin11"); String ascExpected = "min_value\n" + "-------------------------------\n" + "111\n"; @@ -248,7 +248,7 @@ public void testMinLongWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminlongwithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -270,10 +270,10 @@ public void testMaxStringWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("name", TajoDataTypes.Type.TEXT); String[] data = new String[]{ "1|\\N", "2|\\N", "3|\\N" }; - TajoTestingCluster.createTable("testmaxstringwithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select max(name) as max_name from testmaxstringwithnull"); + ResultSet res = executeString("select max(name) as max_name from testbuiltin11"); String ascExpected = "max_name\n" + "-------------------------------\n" + "null\n"; @@ -281,7 +281,7 @@ public void testMaxStringWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testmaxstringwithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -303,10 +303,10 @@ public void testMinStringWithNull() throws Exception { schema.addColumn("id", TajoDataTypes.Type.INT4); schema.addColumn("name", TajoDataTypes.Type.TEXT); String[] data = new String[]{ "1|def", "2|\\N", "3|abc" }; - TajoTestingCluster.createTable("testminstringwithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select min(name) as min_name from testminstringwithnull"); + ResultSet res = executeString("select min(name) as min_name from testbuiltin11"); String ascExpected = "min_name\n" + "-------------------------------\n" + "abc\n"; @@ -314,7 +314,7 @@ public void testMinStringWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testminstringwithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -367,10 +367,10 @@ public void testAvgWithNull() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testavgwithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testavgwithnull"); + ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testbuiltin11"); String ascExpected = "avg_int,avg_long,avg_float,avg_double\n" + "-------------------------------\n" + "1.5,-222.0,2.0,1.0\n"; @@ -378,7 +378,7 @@ public void testAvgWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testavgwithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -396,10 +396,10 @@ public void testAvgWithAllNulls() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|\\N|\\N|\\N", "2|\\N|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testavgwithallnulls", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testavgwithallnulls"); + ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from testbuiltin11"); String ascExpected = "avg_int,avg_long,avg_float,avg_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -407,7 +407,7 @@ public void testAvgWithAllNulls() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testavgwithallnulls PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -425,10 +425,10 @@ public void testSumWithNull() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testsumwithnull", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testsumwithnull"); + ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testbuiltin11"); String ascExpected = "sum_int,sum_long,sum_float,sum_double\n" + "-------------------------------\n" + "3,-444,4.0,2.0\n"; @@ -436,7 +436,7 @@ public void testSumWithNull() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testsumwithnull PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -454,10 +454,10 @@ public void testSumWithAllNulls() throws Exception { schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4); schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8); String[] data = new String[]{ "1|\\N|\\N|\\N|\\N", "2|\\N|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testsumwithallnulls", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testsumwithallnulls"); + ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from testbuiltin11"); String ascExpected = "sum_int,sum_long,sum_float,sum_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -465,7 +465,7 @@ public void testSumWithAllNulls() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testsumwithallnulls PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -486,10 +486,10 @@ public void testStdDevSamp() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("teststddevsamp", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from teststddevsamp"); + ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from testbuiltin11"); String ascExpected = "sdsamp_int,sdsamp_long,sdsamp_float,sdsamp_double\n" + "-------------------------------\n" + "0.7071067811865476,156.97770542341354,1.1313707824635184,72.8319984622144\n"; @@ -497,7 +497,7 @@ public void testStdDevSamp() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE teststddevsamp PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -518,10 +518,10 @@ public void testStdDevSampWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("teststddevsampwithfewnumbers", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from teststddevsampwithfewnumbers"); + ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from testbuiltin11"); String ascExpected = "sdsamp_int,sdsamp_long,sdsamp_float,sdsamp_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -529,7 +529,7 @@ public void testStdDevSampWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE teststddevsampwithfewnumbers PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -550,10 +550,10 @@ public void testStdDevPop() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("teststddevpop", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from teststddevpop"); + ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from testbuiltin11"); String ascExpected = "sdpop_int,sdpop_long,sdpop_float,sdpop_double\n" + "-------------------------------\n" + "0.5,111.0,0.7999999523162842,51.5\n"; @@ -561,7 +561,7 @@ public void testStdDevPop() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE teststddevpop PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -582,10 +582,10 @@ public void testStdDevPopWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("teststddevpopwithfewnumbers", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from teststddevpopwithfewnumbers"); + ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from testbuiltin11"); String ascExpected = "sdpop_int,sdpop_long,sdpop_float,sdpop_double\n" + "-------------------------------\n" + "0.0,null,null,0.0\n"; @@ -593,7 +593,7 @@ public void testStdDevPopWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE teststddevpopwithfewnumbers PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -614,10 +614,10 @@ public void testVarSamp() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testvarsamp", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_samp(value_int) as vs_int, var_samp(value_long) as vs_long, var_samp(value_float) as vs_float, var_samp(value_double) as vs_double from testvarsamp"); + ResultSet res = executeString("select var_samp(value_int) as vs_int, var_samp(value_long) as vs_long, var_samp(value_float) as vs_float, var_samp(value_double) as vs_double from testbuiltin11"); String ascExpected = "vs_int,vs_long,vs_float,vs_double\n" + "-------------------------------\n" + "0.5,24642.0,1.279999847412114,5304.5\n"; @@ -625,7 +625,7 @@ public void testVarSamp() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testvarsamp PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -645,10 +645,10 @@ public void testVarSampWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testvarsampwithfewnumbers", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_samp(value_int) as vsamp_int, var_samp(value_long) as vsamp_long, var_samp(value_float) as vsamp_float, var_samp(value_double) as vsamp_double from testvarsampwithfewnumbers"); + ResultSet res = executeString("select var_samp(value_int) as vsamp_int, var_samp(value_long) as vsamp_long, var_samp(value_float) as vsamp_float, var_samp(value_double) as vsamp_double from testbuiltin11"); String ascExpected = "vsamp_int,vsamp_long,vsamp_float,vsamp_double\n" + "-------------------------------\n" + "null,null,null,null\n"; @@ -656,7 +656,7 @@ public void testVarSampWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testvarsampwithfewnumbers PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -676,10 +676,10 @@ public void testVarPop() throws Exception { "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" }; - TajoTestingCluster.createTable("testvarpop", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testvarpop"); + ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testbuiltin11"); String ascExpected = "vpop_int,vpop_long,vpop_float,vpop_double\n" + "-------------------------------\n" + "0.25,12321.0,0.639999923706057,2652.25\n"; @@ -687,7 +687,7 @@ public void testVarPop() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testvarpop PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -707,10 +707,10 @@ public void testVarPopWithFewNumbers() throws Exception { "1|\\N|\\N|\\N|-50.5", "2|1|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" }; - TajoTestingCluster.createTable("testvarpopwithfewnumbers", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testvarpopwithfewnumbers"); + ResultSet res = executeString("select var_pop(value_int) as vpop_int, var_pop(value_long) as vpop_long, var_pop(value_float) as vpop_float, var_pop(value_double) as vpop_double from testbuiltin11"); String ascExpected = "vpop_int,vpop_long,vpop_float,vpop_double\n" + "-------------------------------\n" + "0.0,null,null,0.0\n"; @@ -718,7 +718,7 @@ public void testVarPopWithFewNumbers() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testvarpopwithfewnumbers PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } @@ -761,16 +761,16 @@ public void testRankWithTwoTables() throws Exception { Schema schema = new Schema(); schema.addColumn("id", TajoDataTypes.Type.INT4); String[] data = new String[] {"1", "3", "2", "4"}; - TajoTestingCluster.createTable("testrankwithtwotables1", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("rank_table1", schema, tableOptions, data, 1); schema = new Schema(); schema.addColumn("refid", TajoDataTypes.Type.INT4); schema.addColumn("value", TajoDataTypes.Type.TEXT); data = new String[] {"1|efgh", "2|abcd", "4|erjk", "8|dfef"}; - TajoTestingCluster.createTable("testrankwithtwotables2", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("rank_table2", schema, tableOptions, data, 1); ResultSet res = null; try { - res = executeString("select rank() over (order by id) from testrankwithtwotables1 a, testrankwithtwotables2 b " + res = executeString("select rank() over (order by id) from rank_table1 a, rank_table2 b " + " where a.id = b.refid"); String expectedString = "?windowfunction\n" + "-------------------------------\n" + @@ -785,8 +785,8 @@ public void testRankWithTwoTables() throws Exception { res.close(); } catch(Throwable ignored) {} } - executeString("DROP TABLE testrankwithtwotables1 PURGE"); - executeString("DROP TABLE testrankwithtwotables2 PURGE"); + executeString("DROP TABLE rank_table1 PURGE"); + executeString("DROP TABLE rank_table2 PURGE"); } } @@ -808,10 +808,10 @@ public void testCorr() throws Exception { "3|2|-333|2.8|\\N", "4|3|-555|2.8|43.2", "5|4|-111|1.1|10.2",}; - TajoTestingCluster.createTable("testcorr", schema, tableOptions, data, 1); + TajoTestingCluster.createTable("testbuiltin11", schema, tableOptions, data, 1); try { - ResultSet res = executeString("select corr(value_int, value_long) as corr1, corr(value_long, value_float) as corr2, corr(value_float, value_double) as corr3, corr(value_double, value_int) as corr4 from testcorr"); + ResultSet res = executeString("select corr(value_int, value_long) as corr1, corr(value_long, value_float) as corr2, corr(value_float, value_double) as corr3, corr(value_double, value_int) as corr4 from testbuiltin11"); String ascExpected = "corr1,corr2,corr3,corr4\n" + "-------------------------------\n" + "0.5,-0.9037045658322675,0.7350290063698216,-0.8761489936497805\n"; @@ -819,7 +819,7 @@ public void testCorr() throws Exception { assertEquals(ascExpected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testcorr PURGE"); + executeString("DROP TABLE testbuiltin11 PURGE"); } } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java index e07d5c5461..bec8cd37e9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java @@ -43,17 +43,17 @@ public void testCaseWhens1() throws IOException { schema.addColumn("col8", CatalogUtil.newDataType(TajoDataTypes.Type.CHAR, "", 3)); schema.addColumn("col9", TajoDataTypes.Type.INT4); - testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", - "select case when col1 between 1 and 3 then 10 else 100 end from testcasewhens1;", + testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", + "select case when col1 between 1 and 3 then 10 else 100 end from table1;", new String [] {"10"}); - testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", - "select case when col1 > 1 then 10 when col1 > 2 then 20 else 100 end from testcasewhens1;", + testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", + "select case when col1 > 1 then 10 when col1 > 2 then 20 else 100 end from table1;", new String [] {"100"}); - testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", - "select case col1 when 1 then 10 when 2 then 20 else 100 end from testcasewhens1;", + testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", + "select case col1 when 1 then 10 when 2 then 20 else 100 end from table1;", new String [] {"10"}); - testEval(schema, "testcasewhens1", "1,2,3,4,5.0,6.0,text,abc,", - "select case col9 when 1 then 10 when 2 then 20 else 100 end is null from testcasewhens1;", + testEval(schema, "table1", "1,2,3,4,5.0,6.0,text,abc,", + "select case col9 when 1 then 10 when 2 then 20 else 100 end is null from table1;", new String [] {"f"}); } @@ -63,11 +63,11 @@ public void testCaseWhensWithNullReturn() throws IOException { schema.addColumn("col1", TajoDataTypes.Type.TEXT); schema.addColumn("col2", TajoDataTypes.Type.TEXT); - testEval(schema, "testcasewhenswithnullreturn", "str1,str2", - "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM testcasewhenswithnullreturn", + testEval(schema, "table1", "str1,str2", + "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM table1", new String[]{"str2"}); - testEval(schema, "testcasewhenswithnullreturn", ",str2", - "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM testcasewhenswithnullreturn", + testEval(schema, "table1", ",str2", + "SELECT CASE WHEN col1 IS NOT NULL THEN col2 ELSE NULL END FROM table1", new String[]{""}); } @@ -78,34 +78,34 @@ public void testCaseWhensWithCommonExpression() throws IOException { schema.addColumn("col2", TajoDataTypes.Type.INT4); schema.addColumn("col3", TajoDataTypes.Type.INT4); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE WHEN col1 = 1 THEN 1 WHEN col1 = 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE WHEN col1 = 1 THEN 1 WHEN col1 = 2 THEN 2 ELSE 3 END FROM table1", new String [] {"1"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE WHEN col2 = 1 THEN 1 WHEN col2 = 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE WHEN col2 = 1 THEN 1 WHEN col2 = 2 THEN 2 ELSE 3 END FROM table1", new String [] {"2"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE WHEN col3 = 1 THEN 1 WHEN col3 = 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE WHEN col3 = 1 THEN 1 WHEN col3 = 2 THEN 2 ELSE 3 END FROM table1", new String [] {"3"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM table1", new String [] {"1"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM table1", new String [] {"2"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE 3 END FROM table1", new String [] {"3"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM table1", new String [] {"aaa"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM table1", new String [] {"bbb"}); - testEval(schema, "testcasewhenswithcommonexpression", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM testcasewhenswithcommonexpression", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN 'aaa' WHEN 2 THEN 'bbb' ELSE 'ccc' END FROM table1", new String [] {"ccc"}); } @@ -116,24 +116,24 @@ public void testCaseWhensWithCommonExpressionAndNull() throws IOException { schema.addColumn("col2", TajoDataTypes.Type.INT4); schema.addColumn("col3", TajoDataTypes.Type.INT4); - testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpressionandnull", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM table1", new String [] {""}); - testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpressionandnull", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM table1", new String [] {"2"}); - testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM testcasewhenswithcommonexpressionandnull", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE 3 END FROM table1", new String [] {"3"}); - testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", - "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE NULL END FROM testcasewhenswithcommonexpressionandnull", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col1 WHEN 1 THEN 1 WHEN 2 THEN 2 ELSE NULL END FROM table1", new String [] {"1"}); - testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", - "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM testcasewhenswithcommonexpressionandnull", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col2 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM table1", new String [] {"2"}); - testEval(schema, "testcasewhenswithcommonexpressionandnull", "1,2,3", - "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM testcasewhenswithcommonexpressionandnull", + testEval(schema, "table1", "1,2,3", + "SELECT CASE col3 WHEN 1 THEN NULL WHEN 2 THEN 2 ELSE NULL END FROM table1", new String [] {""}); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java index 17bc5bce22..25a10fd77c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java @@ -35,6 +35,7 @@ import java.util.TimeZone; import static org.apache.tajo.common.TajoDataTypes.Type.*; +import static org.junit.Assert.assertEquals; public class TestDateTimeFunctions extends ExprTestBase { @Test @@ -121,40 +122,40 @@ public void testExtract() throws IOException { Schema schema2 = new Schema(); schema2.addColumn("col1", TIMESTAMP); - testEval(schema2, "testextract", + testEval(schema2, "table1", "1970-01-17 10:09:37", - "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "testextract", + testEval(schema2, "table1", "1970-01-17 10:09:37" + getUserTimeZoneDisplay(GMT), - "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "testextract", + testEval(schema2, "table1", "1970-01-17 10:09:37" + getUserTimeZoneDisplay(PST), - "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", new String[]{"1970.0", "1.0", "17.0"}); // Currently TIME type can be loaded with INT8 type. Schema schema3 = new Schema(); schema3.addColumn("col1", TIME); - testEval(schema3, "testextract", + testEval(schema3, "table1", "10:09:37.5", - "select extract(hour from col1), extract(minute from col1), extract(second from col1) from testextract;", + "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "testextract", + testEval(schema3, "table1", "10:09:37.5" + getUserTimeZoneDisplay(GMT), - "select extract(hour from col1), extract(minute from col1), extract(second from col1) from testextract;", + "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "testextract", + testEval(schema3, "table1", "10:09:37.5" + getUserTimeZoneDisplay(PST), - "select extract(hour from col1), extract(minute from col1), extract(second from col1) from testextract;", + "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;", new String[]{"18.0", "9.0", "37.5"}); Schema schema4 = new Schema(); schema4.addColumn("col1", DATE); - testEval(schema4, "testextract", + testEval(schema4, "table1", "1970-01-17", - "select extract(year from col1), extract(month from col1), extract(day from col1) from testextract;", + "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", new String[]{"1970.0", "1.0", "17.0"}); testSimpleEval("select extract(century from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"20.0"}); @@ -238,36 +239,36 @@ public void testDatePart() throws IOException { Schema schema2 = new Schema(); schema2.addColumn("col1", TIMESTAMP); - testEval(schema2, "testdatepart", + testEval(schema2, "table1", "1970-01-17 22:09:37", - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "testdatepart", + testEval(schema2, "table1", "1970-01-17 22:09:37" + getUserTimeZoneDisplay(GMT), - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", new String[]{"1970.0", "1.0", "17.0"}); - testEval(schema2, "testdatepart", + testEval(schema2, "table1", "1970-01-17 22:09:37" + getUserTimeZoneDisplay(PST), - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", new String[]{"1970.0", "1.0", "18.0"}); Schema schema3 = new Schema(); schema3.addColumn("col1", TIME); - testEval(schema3, "testdatepart", "10:09:37.5", - "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from testdatepart;", + testEval(schema3, "table1", "10:09:37.5", + "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "testdatepart", "10:09:37.5" + getUserTimeZoneDisplay(GMT), - "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from testdatepart;", + testEval(schema3, "table1", "10:09:37.5" + getUserTimeZoneDisplay(GMT), + "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;", new String[]{"10.0", "9.0", "37.5"}); - testEval(schema3, "testdatepart", "10:09:37.5" + getUserTimeZoneDisplay(PST), - "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from testdatepart;", + testEval(schema3, "table1", "10:09:37.5" + getUserTimeZoneDisplay(PST), + "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;", new String[]{"18.0", "9.0", "37.5"}); Schema schema4 = new Schema(); schema4.addColumn("col1", DATE); - testEval(schema4, "testdatepart", + testEval(schema4, "table1", "1970-01-17", - "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from testdatepart;", + "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", new String[]{"1970.0", "1.0", "17.0"}); testSimpleEval("select date_part('century', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"20.0"}); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java index e0ea6b51a4..78509f77e1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java @@ -42,7 +42,7 @@ public void testRound() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testround", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from testround", + testEval(schema, "table1", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from table1", new String[]{"2"}); Schema schema2 = new Schema(); @@ -51,8 +51,8 @@ public void testRound() throws IOException { schema2.addColumn("col3", FLOAT4); schema2.addColumn("col4", FLOAT8); - testEval(schema2, "testround", "9,9,9.5,9.5", - "select round(col1), round (col2), round(col3), round(col4) from testround", + testEval(schema2, "table1", "9,9,9.5,9.5", + "select round(col1), round (col2), round(col3), round(col4) from table1", new String [] {"9", "9", "10", "10"}); } @@ -70,7 +70,7 @@ public void testFloor() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testfloor", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from testfloor", + testEval(schema, "table1", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from table1", new String[]{"1"}); } @@ -89,7 +89,7 @@ public void testCeil() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testceil", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from testceil", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from table1", new String[]{"2"}); } @@ -108,7 +108,7 @@ public void testCeiling() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testceiling", "1.0, 0.2, 0.1", "select ceiling(col1 + col2 + col3) from testceiling", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceiling(col1 + col2 + col3) from table1", new String[]{"2"}); } @@ -124,7 +124,7 @@ public void testSin() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testsin", "1.0, 0.2, 0.1", "select sin(col1 + col2 + col3) from testsin", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select sin(col1 + col2 + col3) from table1", new String[]{"0.963558185417193"}); } @@ -141,7 +141,7 @@ public void testCos() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testcos", "1.0, 0.2, 0.1", "select cos(col1 + col2 + col3) from testcos", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select cos(col1 + col2 + col3) from table1", new String[]{"0.26749882862458735"}); } @@ -157,7 +157,7 @@ public void testTan() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testtan", "1.0, 0.2, 0.1", "select tan(col1 - col2 - col3) from testtan", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select tan(col1 - col2 - col3) from table1", new String[]{"0.8422883804630795"}); } @@ -173,7 +173,7 @@ public void testAsin() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testasin", "1.0, 0.2, 0.1", "select asin(col1 - col2 - col3) from testasin", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select asin(col1 - col2 - col3) from table1", new String[]{"0.7753974966107532"}); } @@ -189,7 +189,7 @@ public void testAcos() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testAcos", "1.0, 0.2, 0.1", "select acos(col1 - col2 - col3) from testAcos", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select acos(col1 - col2 - col3) from table1", new String[]{"0.7953988301841435"}); } @@ -205,7 +205,7 @@ public void testAtan() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testAtan", "1.0, 0.2, 0.1", "select atan(col1 + col2 + col3) from testAtan", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan(col1 + col2 + col3) from table1", new String[]{"0.9151007005533605"}); } @@ -222,7 +222,7 @@ public void testAtan2() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testAtan2", "1.0, 0.2, 0.1", "select atan2(col1 + col2, col3) from testAtan2", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan2(col1 + col2, col3) from table1", new String[]{"1.4876550949064553"}); } @@ -238,7 +238,7 @@ public void testMod() throws IOException { schema.addColumn("col2", INT8); schema.addColumn("col3", INT8); - testEval(schema, "testMod", "9,2,3", "select mod(col1 + col2, col3) from testMod", + testEval(schema, "table1", "9,2,3", "select mod(col1 + col2, col3) from table1", new String[]{"2"}); } @@ -254,7 +254,7 @@ public void testDiv() throws IOException { schema.addColumn("col2", INT8); schema.addColumn("col3", INT8); - testEval(schema, "testDiv", "9,2,3", "select div(col1 + col2, col3) from testDiv", + testEval(schema, "table1", "9,2,3", "select div(col1 + col2, col3) from table1", new String[]{"3"}); } @@ -279,7 +279,7 @@ public void testSign() throws IOException { schema2.addColumn("col2", FLOAT8); schema2.addColumn("col3", FLOAT8); - testEval(schema2, "testSign", "1.0, 0.2, 0.1", "select sign(col1 + col2 + col3) from testSign", + testEval(schema2, "table1", "1.0, 0.2, 0.1", "select sign(col1 + col2 + col3) from table1", new String[]{"1.0"}); } @@ -295,7 +295,7 @@ public void testSqrt() throws IOException { schema.addColumn("col2", FLOAT4); schema.addColumn("col3", FLOAT4); - testEval(schema, "testSqrt", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from testSqrt", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from table1", new String[]{"1.1401754564651765"}); @@ -304,7 +304,7 @@ public void testSqrt() throws IOException { schema2.addColumn("col2", FLOAT8); schema2.addColumn("col3", FLOAT8); - testEval(schema2, "testSqrt", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from testSqrt", + testEval(schema2, "table1", "1.0, 0.2, 0.1", "select sqrt(col1 + col2 + col3) from table1", new String[]{"1.140175425099138"}); } @@ -318,13 +318,13 @@ public void testExp() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", FLOAT4); - testEval(schema, "testExp", "1.123", "select exp(col1) from testExp", + testEval(schema, "table1", "1.123", "select exp(col1) from table1", new String[]{String.valueOf(Math.exp(1.123f))}); Schema schema2 = new Schema(); schema2.addColumn("col1", FLOAT8); - testEval(schema2, "testExp", "1.123", "select exp(col1) from testExp", + testEval(schema2, "table1", "1.123", "select exp(col1) from table1", new String[]{String.valueOf(Math.exp(1.123d))}); } @@ -341,12 +341,12 @@ public void testAbs() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", FLOAT4); schema.addColumn("col2", FLOAT4); - testEval(schema, "testAbs", "0.39,-0.39", "select abs(col1), abs(col2) from testAbs", new String[]{"0.39", "0.39"}); + testEval(schema, "table1", "0.39,-0.39", "select abs(col1), abs(col2) from table1", new String[]{"0.39", "0.39"}); Schema schema2 = new Schema(); schema2.addColumn("col1", FLOAT8); schema2.addColumn("col2", FLOAT8); - testEval(schema2, "testAbs", "0.033312347,-0.033312347", "select abs(col1), abs(col2) from testAbs", + testEval(schema2, "table1", "0.033312347,-0.033312347", "select abs(col1), abs(col2) from table1", new String[]{"0.033312347", "0.033312347"}); } @@ -361,14 +361,14 @@ public void testCbrt() throws IOException { schema.addColumn("col1", FLOAT4); schema.addColumn("col2", FLOAT4); schema.addColumn("col3", FLOAT4); - testEval(schema, "testCbrt", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from testCbrt", + testEval(schema, "table1", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from table1", new String[]{"1.0913929030771317"}); Schema schema2 = new Schema(); schema2.addColumn("col1", FLOAT8); schema2.addColumn("col2", FLOAT8); schema2.addColumn("col3", FLOAT8); - testEval(schema2, "testCbrt", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from testCbrt", + testEval(schema2, "table1", "1.0, 0.2, 0.1", "select cbrt(col1 + col2 + col3) from table1", new String[]{"1.091392883061106"}); } @@ -384,7 +384,7 @@ public void testDegrees() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testDegrees", "0.8,2.7,-0.8", "select degrees(col1), degrees(col2), degrees(col3) from testDegrees", + testEval(schema, "table1", "0.8,2.7,-0.8", "select degrees(col1), degrees(col2), degrees(col3) from table1", new String[]{ String.valueOf(Math.toDegrees((float)0.8)), String.valueOf(Math.toDegrees(2.7)), @@ -406,7 +406,7 @@ public void testPow() throws IOException { schema.addColumn("col3", INT4); schema.addColumn("col4", INT8); - testEval(schema, "testPow", "0.4,2.7,3,2", "select pow(col1, col2), pow(col3, col4) from testPow", + testEval(schema, "table1", "0.4,2.7,3,2", "select pow(col1, col2), pow(col3, col4) from table1", new String[]{ String.valueOf(Math.pow((float) 0.4, 2.7)), String.valueOf(Math.pow(3, 2)) @@ -425,7 +425,7 @@ public void testRadians() throws IOException { schema.addColumn("col2", FLOAT8); schema.addColumn("col3", FLOAT8); - testEval(schema, "testRadians", "0.8,2.7,-0.8", "select radians(col1), radians(col2), radians(col3) from testRadians", + testEval(schema, "table1", "0.8,2.7,-0.8", "select radians(col1), radians(col2), radians(col3) from table1", new String[]{ String.valueOf(Math.toRadians((float)0.8)), String.valueOf(Math.toRadians(2.7)), @@ -466,6 +466,6 @@ public void testRoundWithSpecifiedPrecision() throws IOException { schema.addColumn("col1", FLOAT8); schema.addColumn("col2", INT4); - testEval(schema, "testRoundWithSpecifiedPrecision", ",", "select round(col1, col2) from testRoundWithSpecifiedPrecision", new String[]{""}); + testEval(schema, "table1", ",", "select round(col1, col2) from table1", new String[]{""}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java index ba51858fb8..8aae26d9c6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java @@ -34,27 +34,27 @@ public void testLike() throws IOException { schema.addColumn("col1", TEXT); // test for null values - testEval(schema, "testlike", ",", "select col1 like 'a%' from testlike", new String[]{""}); + testEval(schema, "table1", ",", "select col1 like 'a%' from table1", new String[]{""}); testSimpleEval("select null like 'a%'", new String[]{""}); - testEval(schema, "testlike", "abc", "select col1 like '%c' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 like 'a%' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 like '_bc' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 like 'ab_' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 like '_b_' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 like '%b%' from testlike", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 like '%c' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 like 'a%' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 like '_bc' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 like 'ab_' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 like '_b_' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 like '%b%' from table1", new String[]{"t"}); // test for escaping regular expressions - testEval(schema, "testlike", "abc", "select col1 not like '.bc' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 not like '.*bc' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 not like '.bc' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 not like '*bc' from testlike", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like '.bc' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like '.*bc' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like '.bc' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like '*bc' from table1", new String[]{"t"}); // test for case sensitive - testEval(schema, "testlike", "abc", "select col1 not like '%C' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 not like 'A%' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 not like '_BC' from testlike", new String[]{"t"}); - testEval(schema, "testlike", "abc", "select col1 not like '_C_' from testlike", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like '%C' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like 'A%' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like '_BC' from table1", new String[]{"t"}); + testEval(schema, "table1", "abc", "select col1 not like '_C_' from table1", new String[]{"t"}); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java index 2727f1faad..7f402a13bc 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java @@ -47,8 +47,8 @@ public void testConcatenateOnExpressions() throws IOException { testSimpleEval("select (1+3) || 2 as col1 ", new String[]{"42"}); - testEval(schema, "testconcatenateonexpressions", "abc,2,3.14", "select col1 || col2 || col3 from testconcatenateonexpressions", new String[]{"abc23.14"}); - testEval(schema, "testconcatenateonexpressions", "abc,2,3.14", "select col1 || '---' || col3 from testconcatenateonexpressions", new String[]{"abc---3.14"}); + testEval(schema, "table1", "abc,2,3.14", "select col1 || col2 || col3 from table1", new String[]{"abc23.14"}); + testEval(schema, "table1", "abc,2,3.14", "select col1 || '---' || col3 from table1", new String[]{"abc---3.14"}); } @Test @@ -72,11 +72,11 @@ public void testLTrim() throws IOException { testSimpleEval("select trim(leading from ' trim') ", new String[]{"trim"}); testSimpleEval("select trim(' trim') ", new String[]{"trim"}); - testEval(schema, "testLTrim", " trim,abc", "select ltrim(col1) from testltrim", new String[]{"trim"}); - testEval(schema, "testLTrim", "xxtrim,abc", "select ltrim(col1, 'xx') from testltrim", new String[]{"trim"}); - testEval(schema, "testLTrim", "xxtrim,abc", "select trim(leading 'xx' from col1) from testltrim", new String[]{"trim"}); + testEval(schema, "table1", " trim,abc", "select ltrim(col1) from table1", new String[]{"trim"}); + testEval(schema, "table1", "xxtrim,abc", "select ltrim(col1, 'xx') from table1", new String[]{"trim"}); + testEval(schema, "table1", "xxtrim,abc", "select trim(leading 'xx' from col1) from table1", new String[]{"trim"}); - testEval(schema, "testLTrim", " trim, abc", "select ltrim(col1) || ltrim(col2) from testltrim", + testEval(schema, "table1", " trim, abc", "select ltrim(col1) || ltrim(col2) from table1", new String[]{"trimabc"}); } @@ -93,11 +93,11 @@ public void testRTrim() throws IOException { testSimpleEval("select trim(trailing from 'trim ') ", new String[]{"trim"}); testSimpleEval("select trim('trim ') ", new String[]{"trim"}); - testEval(schema, "testRTrim", "trim ,abc", "select rtrim(col1) from testRTrim", new String[]{"trim"}); - testEval(schema, "testRTrim", "trimxx,abc", "select rtrim(col1, 'xx') from testRTrim", new String[]{"trim"}); - testEval(schema, "testRTrim", "trimxx,abc", "select trim(trailing 'xx' from col1) from testRTrim", new String[]{"trim"}); + testEval(schema, "table1", "trim ,abc", "select rtrim(col1) from table1", new String[]{"trim"}); + testEval(schema, "table1", "trimxx,abc", "select rtrim(col1, 'xx') from table1", new String[]{"trim"}); + testEval(schema, "table1", "trimxx,abc", "select trim(trailing 'xx' from col1) from table1", new String[]{"trim"}); - testEval(schema, "testRTrim", "trim ,abc ", "select rtrim(col1) || rtrim(col2) from testRTrim", + testEval(schema, "table1", "trim ,abc ", "select rtrim(col1) || rtrim(col2) from table1", new String[]{"trimabc"}); } @@ -114,11 +114,11 @@ public void testTrim() throws IOException { testSimpleEval("select trim(both from ' trim ') ", new String[]{"trim"}); testSimpleEval("select trim(' trim ') ", new String[]{"trim"}); - testEval(schema, "testtrim", " trim ,abc", "select trim(col1) from testtrim", new String[]{"trim"}); - testEval(schema, "testtrim", "xxtrimxx,abc", "select trim(col1, 'xx') from testtrim", new String[]{"trim"}); - testEval(schema, "testtrim", "xxtrimxx,abc", "select trim(both 'xx' from col1) from testtrim", new String[]{"trim"}); + testEval(schema, "table1", " trim ,abc", "select trim(col1) from table1", new String[]{"trim"}); + testEval(schema, "table1", "xxtrimxx,abc", "select trim(col1, 'xx') from table1", new String[]{"trim"}); + testEval(schema, "table1", "xxtrimxx,abc", "select trim(both 'xx' from col1) from table1", new String[]{"trim"}); - testEval(schema, "testtrim", " trim ,xxabcxx", "select trim(col1) || trim(col2,'xx') from testtrim", + testEval(schema, "table1", " trim ,xxabcxx", "select trim(col1) || trim(col2,'xx') from table1", new String[]{"trimabc"}); } @@ -137,13 +137,13 @@ public void testRegexReplace() throws IOException { schema.addColumn("col3", TEXT); // find matches and replace from column values - testEval(schema, "testregexreplace", "------,(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from testregexreplace", + testEval(schema, "table1", "------,(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from table1", new String[]{"ab--ab"}); // null test from a table - testEval(schema, "testregexreplace", ",(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from testregexreplace", + testEval(schema, "table1", ",(^--|--$),ab", "select regexp_replace(col1, col2, col3) as str from table1", new String[]{""}); - testEval(schema, "testregexreplace", "------,(^--|--$),", "select regexp_replace(col1, col2, col3) as str from testregexreplace", + testEval(schema, "table1", "------,(^--|--$),", "select regexp_replace(col1, col2, col3) as str from table1", new String[]{""}); } @@ -170,10 +170,10 @@ public void testLeft() throws IOException { schema.addColumn("col3", TEXT); // for null tests - testEval(schema, "testleft", ",1,ghi", "select left(col1,1) is null from testleft", new String[]{"t"}); - testEval(schema, "testleft", "abc,,ghi", "select left(col1,col2) is null from testleft", new String[]{"t"}); + testEval(schema, "table1", ",1,ghi", "select left(col1,1) is null from table1", new String[]{"t"}); + testEval(schema, "table1", "abc,,ghi", "select left(col1,col2) is null from table1", new String[]{"t"}); - testEval(schema, "testleft", "abc,1,ghi", "select left(col1,1) || left(col3,3) from testleft", new String[]{"aghi"}); + testEval(schema, "table1", "abc,1,ghi", "select left(col1,1) || left(col3,3) from table1", new String[]{"aghi"}); } @Test @@ -199,10 +199,10 @@ public void testRight() throws IOException { schema.addColumn("col3", TEXT); // for null tests - testEval(schema, "testright", ",1,ghi", "select right(col1,1) is null from testright", new String[]{"t"}); - testEval(schema, "testright", "abc,,ghi", "select right(col1,col2) is null from testright", new String[]{"t"}); + testEval(schema, "table1", ",1,ghi", "select right(col1,1) is null from table1", new String[]{"t"}); + testEval(schema, "table1", "abc,,ghi", "select right(col1,col2) is null from table1", new String[]{"t"}); - testEval(schema, "testright", "abc,1,ghi", "select right(col1,1) || right(col3,3) from testright", new String[]{"cghi"}); + testEval(schema, "table1", "abc,1,ghi", "select right(col1,1) || right(col3,3) from table1", new String[]{"cghi"}); } @Test @@ -214,7 +214,7 @@ public void testReverse() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testReverse", "abc,efg,3.14", "select reverse(col1) || reverse(col2) from testReverse", + testEval(schema, "table1", "abc,efg,3.14", "select reverse(col1) || reverse(col2) from table1", new String[]{"cbagfe"}); } @@ -228,7 +228,7 @@ public void testRepeat() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testrepeat", "abc,efg,3.14", "select repeat(col1,2) from testrepeat", new String[]{"abcabc"}); + testEval(schema, "table1", "abc,efg,3.14", "select repeat(col1,2) from table1", new String[]{"abcabc"}); } @@ -240,9 +240,9 @@ public void testUpper() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testupper", "abc,efg,3.14", "select upper(col1), upper(col2) from testupper", + testEval(schema, "table1", "abc,efg,3.14", "select upper(col1), upper(col2) from table1", new String[]{"ABC", "EFG"}); - testEval(schema, "testupper", "abc,efg,3.14", "select upper(col1) || upper(col2) from testupper", new String[]{"ABCEFG"}); + testEval(schema, "table1", "abc,efg,3.14", "select upper(col1) || upper(col2) from table1", new String[]{"ABCEFG"}); } @Test @@ -253,9 +253,9 @@ public void testLower() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testlower", "ABC,DEF,3.14", "select lower(col1), lower(col2) from testlower", + testEval(schema, "table1", "ABC,DEF,3.14", "select lower(col1), lower(col2) from table1", new String[]{"abc", "def"}); - testEval(schema, "testlower", "ABC,DEF,3.14", "select lower(col1) || lower(col2) from testlower", new String[]{"abcdef"}); + testEval(schema, "table1", "ABC,DEF,3.14", "select lower(col1) || lower(col2) from table1", new String[]{"abcdef"}); } @Test @@ -266,7 +266,7 @@ public void testCharLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testcharlength", "ABC,DEF,3.14", "select character_length(lower(col1) || lower(col2)) from testcharlength", + testEval(schema, "table1", "ABC,DEF,3.14", "select character_length(lower(col1) || lower(col2)) from table1", new String[]{"6"}); } @@ -278,7 +278,7 @@ public void testLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testlength", "ABC,DEF,3.14", "select length(lower(col1) || lower(col2)) from testlength", + testEval(schema, "table1", "ABC,DEF,3.14", "select length(lower(col1) || lower(col2)) from table1", new String[]{"6"}); } @@ -291,7 +291,7 @@ public void testMd5() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testmd5", "abc,efg,3.14", "select md5(col1) from testmd5", + testEval(schema, "table1", "abc,efg,3.14", "select md5(col1) from table1", new String[]{"900150983cd24fb0d6963f7d28e17f72"}); } @@ -320,7 +320,7 @@ public void testHex() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testhex", ",abcdef,3.14", "select to_hex(10) from testhex", + testEval(schema, "table1", ",abcdef,3.14", "select to_hex(10) from table1", new String[]{"a"}); } @@ -334,7 +334,7 @@ public void testBin() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testbin", ",abcdef,3.14", "select to_bin(20) from testbin", + testEval(schema, "table1", ",abcdef,3.14", "select to_bin(20) from table1", new String[]{"10100"}); } @@ -348,7 +348,7 @@ public void testOctetLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testoctetlength", "ABC,DEF,3.14", "select octet_length(lower(col1) || lower(col2)) from testoctetlength", + testEval(schema, "table1", "ABC,DEF,3.14", "select octet_length(lower(col1) || lower(col2)) from table1", new String[]{"6"}); } @@ -364,10 +364,10 @@ public void testSplitPart() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testsplitpart", ",.,1", "select split_part(col1, col2, col3::int) is null from testsplitpart", new String[]{"t"}); - testEval(schema, "testsplitpart", "1386577650.123,,1", "select split_part(col1, col2, col3::int) from testsplitpart", + testEval(schema, "t1", ",.,1", "select split_part(col1, col2, col3::int) is null from t1", new String[]{"t"}); + testEval(schema, "t1", "1386577650.123,,1", "select split_part(col1, col2, col3::int) from t1", new String[]{"1386577650.123"}); - testEval(schema, "testsplitpart", "1386577650.123,.,", "select split_part(col1, col2, col3::int) is null from testsplitpart", + testEval(schema, "t1", "1386577650.123,.,", "select split_part(col1, col2, col3::int) is null from t1", new String[]{"t"}); } @@ -393,7 +393,7 @@ public void testSubstr() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testsubstr", ",abcdef,3.14", "select substr(lower(col2), 2, 3) from testsubstr", + testEval(schema, "table1", ",abcdef,3.14", "select substr(lower(col2), 2, 3) from table1", new String[]{"bcd"}); } @@ -436,14 +436,14 @@ public void testLocate() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'cd') from testlocate", new String[]{"3"}); - testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'cd', 1) from testlocate", new String[]{"3"}); - testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'cd', 4) from testlocate", new String[]{"0"}); - testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col2, 'xy') from testlocate", new String[]{"0"}); + testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd') from table1", new String[]{"3"}); + testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd', 1) from table1", new String[]{"3"}); + testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'cd', 4) from table1", new String[]{"0"}); + testEval(schema, "table1", ",abcdef,3.14", "select locate(col2, 'xy') from table1", new String[]{"0"}); // null string - testEval(schema, "testlocate", ",abcdef,3.14", "select locate(col1, 'cd') is null from testlocate", new String[]{"t"}); + testEval(schema, "table1", ",abcdef,3.14", "select locate(col1, 'cd') is null from table1", new String[]{"t"}); // nul substring - testEval(schema, "testlocate", ",abcdef,3.14", "select locate('cd', col1) is null from testlocate", new String[]{"t"}); + testEval(schema, "table1", ",abcdef,3.14", "select locate('cd', col1) is null from table1", new String[]{"t"}); } @Test @@ -454,7 +454,7 @@ public void testBitLength() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "testbitlength", "ABC,DEF,3.14", "select bit_length(lower(col1) || lower(col2)) from testbitlength", + testEval(schema, "table1", "ABC,DEF,3.14", "select bit_length(lower(col1) || lower(col2)) from table1", new String[]{"48"}); } @@ -470,7 +470,7 @@ public void testStrpos() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "teststrpos", "ABCDEF,HIJKLMN,3.14", "select strpos(lower(col1) || lower(col2), 'fh') from teststrpos", + testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strpos(lower(col1) || lower(col2), 'fh') from table1", new String[]{"6"}); } @@ -486,7 +486,7 @@ public void testStrposb() throws IOException { schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); schema.addColumn("col3", TEXT); - testEval(schema, "teststrposb", "ABCDEF,HIJKLMN,3.14", "select strposb(lower(col1) || lower(col2), 'fh') from teststrposb", + testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strposb(lower(col1) || lower(col2), 'fh') from table1", new String[]{"6"}); } @@ -502,9 +502,9 @@ public void testAscii() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TEXT); - testEval(schema, "testascii", "abc", "select ascii(col1) from testascii", + testEval(schema, "table1", "abc", "select ascii(col1) from table1", new String[]{"97"}); - testEval(schema, "testascii", "12", "select ascii(col1) from testascii", + testEval(schema, "table1", "12", "select ascii(col1) from table1", new String[]{"49"}); } @@ -518,9 +518,9 @@ public void testChr() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", INT4); - testEval(schema, "testchr", "65", "select chr(col1) from testchr", new String[]{"A"}); - testEval(schema, "testchr", "66", "select chr(col1) from testchr", new String[]{"B"}); - testEval(schema, "testchr", "52512", "select chr(col1) from testchr", new String[]{"ì´ "}); + testEval(schema, "table1", "65", "select chr(col1) from table1", new String[]{"A"}); + testEval(schema, "table1", "66", "select chr(col1) from table1", new String[]{"B"}); + testEval(schema, "table1", "52512", "select chr(col1) from table1", new String[]{"ì´ "}); } @Test @@ -590,9 +590,9 @@ public void testFindInSet() throws IOException { Schema schema = new Schema(); schema.addColumn("col1", TEXT); schema.addColumn("col2", TEXT); - testEval(schema, "testfindinset", "|crt,c,cr,c,def", "select find_in_set(col1, col2) is null from testfindinset", + testEval(schema, "table1", "|crt,c,cr,c,def", "select find_in_set(col1, col2) is null from table1", new String[]{"t"}, '|', true); - testEval(schema, "testfindinset", "cr|", "select find_in_set(col1, col2) is null from testfindinset", + testEval(schema, "table1", "cr|", "select find_in_set(col1, col2) is null from table1", new String[]{"t"}, '|', true); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java index 0e9eb074b0..ace3d0d901 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -29,53 +28,54 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.PhysicalPlanner; -import org.apache.tajo.engine.planner.PhysicalPlannerImpl; +import org.apache.tajo.engine.planner.*; import org.apache.tajo.engine.planner.enforce.Enforcer; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; +import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class TestBNLJoinExec { - private static TajoConf conf; - private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestBNLJoinExec"; - private static TajoTestingCluster util; - private static CatalogService catalog; - private static LogicalPlanner planner; - private static Path testDir; + private TajoConf conf; + private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestBNLJoinExec"; + private TajoTestingCluster util; + private CatalogService catalog; + private SQLAnalyzer analyzer; + private LogicalPlanner planner; + private Path testDir; private static int OUTER_TUPLE_NUM = 1000; private static int INNER_TUPLE_NUM = 1000; - private static TableDesc employee; - private static TableDesc people; + private TableDesc employee; + private TableDesc people; - @BeforeClass - public static void setUp() throws Exception { - util = TpchTestBase.getInstance().getTestingCluster(); + @Before + public void setUp() throws Exception { + util = new TajoTestingCluster(); catalog = util.startCatalogCluster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); - catalog.createDatabase("testbnljoinexec", DEFAULT_TABLESPACE_NAME); + catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); Schema schema = new Schema(); @@ -98,7 +98,7 @@ public static void setUp() throws Exception { } appender.flush(); appender.close(); - employee = CatalogUtil.newTableDesc("testbnljoinexec.employee", schema, employeeMeta, employeePath); + employee = CatalogUtil.newTableDesc("default.employee", schema, employeeMeta, employeePath); catalog.createTable(employee); Schema peopleSchema = new Schema(); @@ -121,27 +121,26 @@ public static void setUp() throws Exception { appender.flush(); appender.close(); - people = CatalogUtil.newTableDesc("testbnljoinexec.people", peopleSchema, peopleMeta, peoplePath); + people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath); catalog.createTable(people); + analyzer = new SQLAnalyzer(); planner = new LogicalPlanner(catalog, TablespaceManager.getInstance()); } - @AfterClass - public static void tearDown() throws Exception { - catalog.dropDatabase("testbnljoinexec"); - testDir.getFileSystem(conf).delete(testDir, true); + @After + public void tearDown() throws Exception { + util.shutdownCatalogCluster(); } // employee (managerId, empId, memId, deptName) // people (empId, fk_memId, name, age) String[] QUERIES = { - "select managerId, e.empId, deptName, e.memId from testbnljoinexec.employee as e, testbnljoinexec.people p", - "select managerId, e.empId, deptName, e.memId from testbnljoinexec.employee as e " + - "inner join testbnljoinexec.people as p on e.empId = p.empId and e.memId = p.fk_memId" }; + "select managerId, e.empId, deptName, e.memId from employee as e, people p", + "select managerId, e.empId, deptName, e.memId from employee as e " + + "inner join people as p on e.empId = p.empId and e.memId = p.fk_memId" }; @Test public final void testBNLCrossJoin() throws IOException, PlanningException { - SQLAnalyzer analyzer = new SQLAnalyzer(); Expr expr = analyzer.parse(QUERIES[0]); LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr).getRootBlock().getRoot(); @@ -149,10 +148,10 @@ public final void testBNLCrossJoin() throws IOException, PlanningException { Enforcer enforcer = new Enforcer(); enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN); - FileFragment[] empFrags = FileTablespace.splitNG(conf, "testbnljoinexec.e", employee.getMeta(), + FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(), new Path(employee.getUri()), Integer.MAX_VALUE); - FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "testbnljoinexec.p", people.getMeta(), + FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(), new Path(people.getUri()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(empFrags, peopleFrags); @@ -178,14 +177,13 @@ public final void testBNLCrossJoin() throws IOException, PlanningException { @Test public final void testBNLInnerJoin() throws IOException, PlanningException { - SQLAnalyzer analyzer = new SQLAnalyzer(); Expr context = analyzer.parse(QUERIES[1]); LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), context).getRootBlock().getRoot(); - FileFragment[] empFrags = FileTablespace.splitNG(conf, "testbnljoinexec.e", employee.getMeta(), + FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(), new Path(employee.getUri()), Integer.MAX_VALUE); - FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "testbnljoinexec.p", people.getMeta(), + FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(), new Path(people.getUri()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(empFrags, peopleFrags); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java index 3ad14c349a..dc4dd042b8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -29,61 +28,59 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.PhysicalPlanner; -import org.apache.tajo.engine.planner.PhysicalPlannerImpl; +import org.apache.tajo.engine.planner.*; import org.apache.tajo.engine.planner.enforce.Enforcer; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.storage.Appender; -import org.apache.tajo.storage.FileTablespace; -import org.apache.tajo.storage.TablespaceManager; -import org.apache.tajo.storage.VTuple; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; +import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestFullOuterHashJoinExec { - private static TajoConf conf; - private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterHashJoinExec"; - private static TajoTestingCluster util; - private static CatalogService catalog; - private static SQLAnalyzer analyzer; - private static LogicalPlanner planner; - private static Path testDir; - private static QueryContext defaultContext; - - private static TableDesc dep3; - private static TableDesc job3; - private static TableDesc emp3; - private static TableDesc phone3; - - private static final String DATABASENAME = "testfullouterhashjoinexec"; - private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); - private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); - private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); - private static final String PHONE3_NAME = CatalogUtil.buildFQName(DATABASENAME, "phone3"); - - @BeforeClass - public static void setUp() throws Exception { - util = TpchTestBase.getInstance().getTestingCluster(); - catalog = util.getMaster().getCatalog(); + private TajoConf conf; + private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterHashJoinExec"; + private TajoTestingCluster util; + private CatalogService catalog; + private SQLAnalyzer analyzer; + private LogicalPlanner planner; + private Path testDir; + private QueryContext defaultContext; + + private TableDesc dep3; + private TableDesc job3; + private TableDesc emp3; + private TableDesc phone3; + + private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); + private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); + private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); + private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); + + @Before + public void setUp() throws Exception { + util = new TajoTestingCluster(); + util.initTestDir(); + catalog = util.startCatalogCluster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); + catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); + catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -242,25 +239,20 @@ public static void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @AfterClass - public static void tearDown() throws Exception { - catalog.dropDatabase(DATABASENAME); - testDir.getFileSystem(conf).delete(testDir, true); + @After + public void tearDown() throws Exception { + util.shutdownCatalogCluster(); } String[] QUERIES = { // [0] no nulls - String.format("select dep3.dep_id, dep_name, emp_id, salary from %s.dep3 full outer join %s.emp3 on %s.dep3.dep_id = %s.emp3.dep_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select dep3.dep_id, dep_name, emp_id, salary from dep3 full outer join emp3 on dep3.dep_id = emp3.dep_id", // [1] nulls on the right operand - String.format("select job3.job_id, job_title, emp_id, salary from %s.job3 full outer join %s.emp3 on %s.job3.job_id=%s.emp3.job_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select job3.job_id, job_title, emp_id, salary from job3 full outer join emp3 on job3.job_id=emp3.job_id", // [2] nulls on the left side - String.format("select job3.job_id, job_title, emp_id, salary from %s.emp3 full outer join %s.job3 on %s.job3.job_id=%s.emp3.job_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select job3.job_id, job_title, emp_id, salary from emp3 full outer join job3 on job3.job_id=emp3.job_id", // [3] one operand is empty - String.format("select emp3.emp_id, first_name, phone_number from %s.emp3 full outer join %s.phone3 on %s.emp3.emp_id = %s.phone3.emp_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select emp3.emp_id, first_name, phone_number from emp3 full outer join phone3 on emp3.emp_id = phone3.emp_id" }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java index 061cd85292..8fd61d0de0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java @@ -20,8 +20,8 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; +import org.apache.tajo.TajoConstants; import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -39,53 +39,53 @@ import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.storage.Appender; -import org.apache.tajo.storage.FileTablespace; -import org.apache.tajo.storage.TablespaceManager; -import org.apache.tajo.storage.VTuple; +import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; +import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestFullOuterMergeJoinExec { - private static TajoConf conf; - private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterMergeJoinExec"; - private static TajoTestingCluster util; - private static CatalogService catalog; - private static SQLAnalyzer analyzer; - private static LogicalPlanner planner; - private static Path testDir; - private static QueryContext defaultContext; - - private static TableDesc dep3; - private static TableDesc dep4; - private static TableDesc job3; - private static TableDesc emp3; - private static TableDesc phone3; - - private static final String DATABASENAME = "testfulloutermergejoinexec"; - private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); - private static final String DEP4_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep4"); - private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); - private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); - private static final String PHONE3_NAME = CatalogUtil.buildFQName(DATABASENAME, "phone3"); - - @BeforeClass - public static void setUp() throws Exception { - util = TpchTestBase.getInstance().getTestingCluster(); - catalog = util.getMaster().getCatalog(); + private TajoConf conf; + private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestFullOuterMergeJoinExec"; + private TajoTestingCluster util; + private CatalogService catalog; + private SQLAnalyzer analyzer; + private LogicalPlanner planner; + private Path testDir; + private QueryContext defaultContext; + + private TableDesc dep3; + private TableDesc dep4; + private TableDesc job3; + private TableDesc emp3; + private TableDesc phone3; + + private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); + private final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4"); + private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); + private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); + private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); + + @Before + public void setUp() throws Exception { + util = new TajoTestingCluster(); + util.initTestDir(); + catalog = util.startCatalogCluster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); + catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); + catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); + conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -283,32 +283,24 @@ public static void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - - @AfterClass - public static void tearDown() throws Exception { - catalog.dropDatabase(DATABASENAME); - testDir.getFileSystem(conf).delete(testDir, true); + @After + public void tearDown() throws Exception { + util.shutdownCatalogCluster(); } String[] QUERIES = { // [0] no nulls - String.format("select %s.dep3.dep_id, dep_name, emp_id, salary from %s.emp3 full outer join %s.dep3 on %s.dep3.dep_id = %s.emp3.dep_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select dep3.dep_id, dep_name, emp_id, salary from emp3 full outer join dep3 on dep3.dep_id = emp3.dep_id", // [1] nulls on the left operand - String.format("select %s.job3.job_id, job_title, emp_id, salary from %s.emp3 full outer join %s.job3 on %s.job3.job_id=%s.emp3.job_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select job3.job_id, job_title, emp_id, salary from emp3 full outer join job3 on job3.job_id=emp3.job_id", // [2] nulls on the right side - String.format("select %s.job3.job_id, job_title, emp_id, salary from %s.job3 full outer join %s.emp3 on %s.job3.job_id=%s.emp3.job_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select job3.job_id, job_title, emp_id, salary from job3 full outer join emp3 on job3.job_id=emp3.job_id", // [3] no nulls, right continues after left - String.format("select %s.dep4.dep_id, dep_name, emp_id, salary from %s.emp3 full outer join %s.dep4 on %s.dep4.dep_id = %s.emp3.dep_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select dep4.dep_id, dep_name, emp_id, salary from emp3 full outer join dep4 on dep4.dep_id = emp3.dep_id", // [4] one operand is empty - String.format("select %s.emp3.emp_id, first_name, phone_number from %s.emp3 full outer join %s.phone3 on %s.emp3.emp_id = %s.phone3.emp_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select emp3.emp_id, first_name, phone_number from emp3 full outer join phone3 on emp3.emp_id = phone3.emp_id", // [5] one operand is empty - String.format("select %s.emp3.emp_id, first_name, phone_number from %s.phone3 full outer join %s.emp3 on %s.emp3.emp_id = %s.phone3.emp_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select emp3.emp_id, first_name, phone_number from phone3 full outer join emp3 on emp3.emp_id = phone3.emp_id", }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java index ca5dbbc3f9..c93a1b40b6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -29,60 +28,60 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.PhysicalPlanner; -import org.apache.tajo.engine.planner.PhysicalPlannerImpl; +import org.apache.tajo.engine.planner.*; import org.apache.tajo.engine.planner.enforce.Enforcer; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; +import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class TestLeftOuterHashJoinExec { - - private static TajoConf conf; - private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuterHashJoinExec"; - private static TajoTestingCluster util; - private static CatalogService catalog; - private static SQLAnalyzer analyzer; - private static LogicalPlanner planner; - private static Path testDir; - private static QueryContext defaultContext; - - private static TableDesc dep3; - private static TableDesc job3; - private static TableDesc emp3; - private static TableDesc phone3; - - private static final String DATABASENAME = "testleftouterhashjoinexec"; - private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); - private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); - private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); - private static final String PHONE3_NAME = CatalogUtil.buildFQName(DATABASENAME, "phone3"); - - @BeforeClass - public static void setUp() throws Exception { - util = TpchTestBase.getInstance().getTestingCluster(); - catalog = util.getMaster().getCatalog(); + private TajoConf conf; + private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuterHashJoinExec"; + private TajoTestingCluster util; + private CatalogService catalog; + private SQLAnalyzer analyzer; + private LogicalPlanner planner; + private Path testDir; + private QueryContext defaultContext; + + private TableDesc dep3; + private TableDesc job3; + private TableDesc emp3; + private TableDesc phone3; + + private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); + private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); + private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); + private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); + + @Before + public void setUp() throws Exception { + util = new TajoTestingCluster(); + util.initTestDir(); + catalog = util.startCatalogCluster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); + catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); + catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -244,28 +243,22 @@ public static void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @AfterClass - public static void tearDown() throws Exception { - catalog.dropDatabase(DATABASENAME); - testDir.getFileSystem(conf).delete(testDir, true); + @After + public void tearDown() throws Exception { + util.shutdownCatalogCluster(); } String[] QUERIES = { // [0] no nulls - String.format("select dep3.dep_id, dep_name, emp_id, salary from %s.dep3 left outer join %s.emp3 on %s.dep3.dep_id = %s.emp3.dep_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select dep3.dep_id, dep_name, emp_id, salary from dep3 left outer join emp3 on dep3.dep_id = emp3.dep_id", // [1] nulls on the right operand - String.format("select job3.job_id, job_title, emp_id, salary from %s.job3 left outer join %s.emp3 on %s.job3.job_id=%s.emp3.job_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select job3.job_id, job_title, emp_id, salary from job3 left outer join emp3 on job3.job_id=emp3.job_id", // [2] nulls on the left side - String.format("select job3.job_id, job_title, emp_id, salary from %s.emp3 left outer join %s.job3 on %s.job3.job_id=%s.emp3.job_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select job3.job_id, job_title, emp_id, salary from emp3 left outer join job3 on job3.job_id=emp3.job_id", // [3] one operand is empty - String.format("select emp3.emp_id, first_name, phone_number from %s.emp3 left outer join %s.phone3 on %s.emp3.emp_id = %s.phone3.emp_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select emp3.emp_id, first_name, phone_number from emp3 left outer join phone3 on emp3.emp_id = phone3.emp_id", // [4] one operand is empty - String.format("select phone_number, emp3.emp_id, first_name from %s.phone3 left outer join %s.emp3 on %s.emp3.emp_id = %s.phone3.emp_id", - DATABASENAME, DATABASENAME, DATABASENAME, DATABASENAME), + "select phone_number, emp3.emp_id, first_name from phone3 left outer join emp3 on emp3.emp_id = phone3.emp_id" }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java index 3a13342001..f581db86de 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.TajoTestingCluster; -import org.apache.tajo.TpchTestBase; import org.apache.tajo.algebra.Expr; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; @@ -41,42 +40,43 @@ import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; +import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; import static org.junit.Assert.assertEquals; // this is not a physical operator in itself, but it uses the HashLeftOuterJoinExec with switched inputs order public class TestRightOuterHashJoinExec { - - private static TajoConf conf; - private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterHashJoinExec"; - private static TajoTestingCluster util; - private static CatalogService catalog; - private static SQLAnalyzer analyzer; - private static LogicalPlanner planner; - private static Path testDir; - private static QueryContext defaultContext; - - private static TableDesc dep3; - private static TableDesc job3; - private static TableDesc emp3; - - private static final String DATABASENAME = "testrightouterhashjoinexec"; - private static final String DEP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "dep3"); - private static final String JOB3_NAME = CatalogUtil.buildFQName(DATABASENAME, "job3"); - private static final String EMP3_NAME = CatalogUtil.buildFQName(DATABASENAME, "emp3"); - - @BeforeClass - public static void setUp() throws Exception { - util = TpchTestBase.getInstance().getTestingCluster(); - catalog = util.getMaster().getCatalog(); + private TajoConf conf; + private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterHashJoinExec"; + private TajoTestingCluster util; + private CatalogService catalog; + private SQLAnalyzer analyzer; + private LogicalPlanner planner; + private Path testDir; + private QueryContext defaultContext; + + private TableDesc dep3; + private TableDesc job3; + private TableDesc emp3; + + private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); + private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); + private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); + + @Before + public void setUp() throws Exception { + util = new TajoTestingCluster(); + util.initTestDir(); + catalog = util.startCatalogCluster().getCatalog(); testDir = CommonTestingUtil.getTestDir(TEST_PATH); - catalog.createDatabase(DATABASENAME, DEFAULT_TABLESPACE_NAME); + catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString()); + catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); conf = util.getConfiguration(); //----------------- dep3 ------------------------------ @@ -216,22 +216,15 @@ public static void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @AfterClass - public static void tearDown() throws Exception { - catalog.dropDatabase(DATABASENAME); - testDir.getFileSystem(conf).delete(testDir, true); + @After + public void tearDown() throws Exception { + util.shutdownCatalogCluster(); } String[] QUERIES = { - //0 no nulls - String.format("select dep3.dep_id, dep_name, emp_id, salary from %s.emp3 right outer join %s.dep3 on dep3.dep_id = emp3.dep_id", - DATABASENAME, DATABASENAME), - //1 nulls on the left operand - String.format("select job3.job_id, job_title, emp_id, salary from %s.emp3 right outer join %s.job3 on job3.job_id=emp3.job_id", - DATABASENAME, DATABASENAME), - //2 nulls on the right side - String.format("select job3.job_id, job_title, emp_id, salary from %s.job3 right outer join %s.emp3 on job3.job_id=emp3.job_id", - DATABASENAME, DATABASENAME) + "select dep3.dep_id, dep_name, emp_id, salary from emp3 right outer join dep3 on dep3.dep_id = emp3.dep_id", //0 no nulls + "select job3.job_id, job_title, emp_id, salary from emp3 right outer join job3 on job3.job_id=emp3.job_id", //1 nulls on the left operand + "select job3.job_id, job_title, emp_id, salary from job3 right outer join emp3 on job3.job_id=emp3.job_id" //2 nulls on the right side }; @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java index 23b95ff18d..d86b2294bb 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java @@ -28,26 +28,22 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.planner.PhysicalPlanner; -import org.apache.tajo.engine.planner.PhysicalPlannerImpl; +import org.apache.tajo.engine.planner.*; import org.apache.tajo.engine.planner.enforce.Enforcer; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.logical.JoinNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.storage.Appender; -import org.apache.tajo.storage.FileTablespace; -import org.apache.tajo.storage.TablespaceManager; -import org.apache.tajo.storage.VTuple; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.TaskAttemptContext; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.IOException; @@ -58,30 +54,30 @@ import static org.junit.Assert.*; public class TestRightOuterMergeJoinExec { - private static TajoConf conf; - private static final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterMergeJoinExec"; - private static TajoTestingCluster util; - private static CatalogService catalog; - private static SQLAnalyzer analyzer; - private static LogicalPlanner planner; - private static Path testDir; - private static QueryContext defaultContext; - - - private static TableDesc dep3; - private static TableDesc dep4; - private static TableDesc job3; - private static TableDesc emp3; - private static TableDesc phone3; - - private static final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); - private static final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4"); - private static final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); - private static final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); - private static final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); - - @BeforeClass - public static void setUp() throws Exception { + private TajoConf conf; + private final String TEST_PATH = TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestRightOuterMergeJoinExec"; + private TajoTestingCluster util; + private CatalogService catalog; + private SQLAnalyzer analyzer; + private LogicalPlanner planner; + private Path testDir; + private QueryContext defaultContext; + + + private TableDesc dep3; + private TableDesc dep4; + private TableDesc job3; + private TableDesc emp3; + private TableDesc phone3; + + private final String DEP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep3"); + private final String DEP4_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "dep4"); + private final String JOB3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "job3"); + private final String EMP3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "emp3"); + private final String PHONE3_NAME = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "phone3"); + + @Before + public void setUp() throws Exception { util = new TajoTestingCluster(); util.initTestDir(); catalog = util.startCatalogCluster().getCatalog(); @@ -290,8 +286,8 @@ public static void setUp() throws Exception { defaultContext = LocalTajoTestingUtility.createDummyContext(conf); } - @AfterClass - public static void tearDown() throws Exception { + @After + public void tearDown() throws Exception { util.shutdownCatalogCluster(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java index 86912fd12c..539a4a4f16 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java @@ -59,35 +59,35 @@ public final void testCreateTable1() throws Exception { @Test public final void testCreateTable2() throws Exception { - executeString("CREATE DATABASE testcreatetable2;").close(); - executeString("CREATE DATABASE testcreatetable2_2;").close(); - - executeString("CREATE TABLE testcreatetable2.table1 (age int);").close(); - executeString("CREATE TABLE testcreatetable2.table2 (age int);").close(); - executeString("CREATE TABLE testcreatetable2_2.table3 (age int);").close(); - executeString("CREATE TABLE testcreatetable2_2.table4 (age int);").close(); - - assertTableExists("testcreatetable2.table1"); - assertTableExists("testcreatetable2.table2"); - assertTableNotExists("testcreatetable2_2.table1"); - assertTableNotExists("testcreatetable2_2.table2"); - - assertTableExists("testcreatetable2_2.table3"); - assertTableExists("testcreatetable2_2.table4"); - assertTableNotExists("testcreatetable2.table3"); - assertTableNotExists("testcreatetable2.table4"); - - executeString("DROP TABLE testcreatetable2.table1"); - executeString("DROP TABLE testcreatetable2.table2"); - executeString("DROP TABLE testcreatetable2_2.table3"); - executeString("DROP TABLE testcreatetable2_2.table4"); - - assertDatabaseExists("testcreatetable2"); - assertDatabaseExists("testcreatetable2_2"); - executeString("DROP DATABASE testcreatetable2").close(); - executeString("DROP DATABASE testcreatetable2_2").close(); - assertDatabaseNotExists("testcreatetable2"); - assertDatabaseNotExists("testcreatetable2_2"); + executeString("CREATE DATABASE D1;").close(); + executeString("CREATE DATABASE D2;").close(); + + executeString("CREATE TABLE D1.table1 (age int);").close(); + executeString("CREATE TABLE D1.table2 (age int);").close(); + executeString("CREATE TABLE d2.table3 (age int);").close(); + executeString("CREATE TABLE d2.table4 (age int);").close(); + + assertTableExists("d1.table1"); + assertTableExists("d1.table2"); + assertTableNotExists("d2.table1"); + assertTableNotExists("d2.table2"); + + assertTableExists("d2.table3"); + assertTableExists("d2.table4"); + assertTableNotExists("d1.table3"); + assertTableNotExists("d1.table4"); + + executeString("DROP TABLE D1.table1"); + executeString("DROP TABLE D1.table2"); + executeString("DROP TABLE D2.table3"); + executeString("DROP TABLE D2.table4"); + + assertDatabaseExists("d1"); + assertDatabaseExists("d2"); + executeString("DROP DATABASE D1").close(); + executeString("DROP DATABASE D2").close(); + assertDatabaseNotExists("d1"); + assertDatabaseNotExists("d2"); } private final void assertPathOfCreatedTable(final String databaseName, @@ -212,15 +212,13 @@ public final void testDelimitedIdentifierWithMixedCharacters() throws Exception ResultSet res = null; try { - List tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table1", - "\"TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS\""); + List tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table1", "\"TABLE1\""); assertTableExists(tableNames.get(0)); - tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table2", - "\"testDelimitedIdentifierWithMixedCharacters\""); + tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table2", "\"tablE1\""); assertTableExists(tableNames.get(0)); - // SELECT "aGe", "tExt", "Number" FROM "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS"; + // SELECT "aGe", "tExt", "Number" FROM "TABLE1"; res = executeFile("quoted_identifier_mixed_chars_1.sql"); assertResultSet(res, "quoted_identifier_mixed_chars_1.result"); } finally { @@ -606,25 +604,25 @@ public final void testCreateTableLike1() throws Exception { @Test public final void testNestedRecord1() throws Exception { - executeString("CREATE DATABASE d9;").close(); + executeString("CREATE DATABASE D9;").close(); assertTableNotExists("d9.nested_table"); executeQuery().close(); assertTableExists("d9.nested_table"); - executeString("DROP TABLE d9.nested_table"); - executeString("DROP DATABASE d9").close(); + executeString("DROP TABLE D9.nested_table"); + executeString("DROP DATABASE D9").close(); } @Test public final void testNestedRecord2() throws Exception { - executeString("CREATE DATABASE d10;").close(); + executeString("CREATE DATABASE D9;").close(); - assertTableNotExists("d10.nested_table2"); + assertTableNotExists("d9.nested_table2"); executeQuery(); - assertTableExists("d10.nested_table2"); + assertTableExists("d9.nested_table2"); - executeString("DROP TABLE d10.nested_table2"); - executeString("DROP DATABASE d10").close(); + executeString("DROP TABLE D9.nested_table2"); + executeString("DROP DATABASE D9").close(); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java index 04d64db819..6adcc1fa56 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java @@ -23,10 +23,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.*; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter; @@ -41,8 +38,8 @@ import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.logical.ScanNode; import org.apache.tajo.storage.StorageConstants; -import org.apache.tajo.storage.Tablespace; import org.apache.tajo.storage.TablespaceManager; +import org.apache.tajo.storage.Tablespace; import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.storage.hbase.*; import org.apache.tajo.util.Bytes; @@ -58,12 +55,10 @@ import java.net.URI; import java.sql.ResultSet; import java.text.DecimalFormat; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) @net.jcip.annotations.NotThreadSafe @@ -103,7 +98,7 @@ public static void afterClass() { @Test public void testVerifyCreateHBaseTableRequiredMeta() throws Exception { try { - executeString("CREATE TABLE testverifycreatehbasetablerequiredmeta (col1 text, col2 text) TABLESPACE cluster1 USING hbase").close(); + executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) TABLESPACE cluster1 USING hbase").close(); fail("hbase table must have 'table' meta"); } catch (Exception e) { @@ -111,9 +106,9 @@ public void testVerifyCreateHBaseTableRequiredMeta() throws Exception { } try { - executeString("CREATE TABLE testverifycreatehbasetablerequiredmeta (col1 text, col2 text) TABLESPACE cluster1 " + + executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) TABLESPACE cluster1 " + "USING hbase " + - "WITH ('table'='testverifycreatehbasetablerequiredmeta')").close(); + "WITH ('table'='hbase_table')").close(); fail("hbase table must have 'columns' meta"); } catch (Exception e) { @@ -124,14 +119,14 @@ public void testVerifyCreateHBaseTableRequiredMeta() throws Exception { @Test public void testCreateHBaseTable() throws Exception { executeString( - "CREATE TABLE testcreatehbasetable (col1 text, col2 text, col3 text, col4 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testcreatehbasetable', 'columns'=':key,col2:a,col3:,col2:b')").close(); + "CREATE TABLE hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:a,col3:,col2:b')").close(); - assertTableExists("testcreatehbasetable"); + assertTableExists("hbase_mapped_table1"); - HTableDescriptor hTableDesc = testingCluster.getHBaseUtil().getTableDescriptor("testcreatehbasetable"); + HTableDescriptor hTableDesc = testingCluster.getHBaseUtil().getTableDescriptor("hbase_table"); assertNotNull(hTableDesc); - assertEquals("testcreatehbasetable", hTableDesc.getNameAsString()); + assertEquals("hbase_table", hTableDesc.getNameAsString()); HColumnDescriptor[] hColumns = hTableDesc.getColumnFamilies(); // col1 is mapped to rowkey @@ -139,11 +134,11 @@ public void testCreateHBaseTable() throws Exception { assertEquals("col2", hColumns[0].getNameAsString()); assertEquals("col3", hColumns[1].getNameAsString()); - executeString("DROP TABLE testcreatehbasetable PURGE").close(); + executeString("DROP TABLE hbase_mapped_table1 PURGE").close(); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); try { - assertFalse(hAdmin.tableExists("testcreatehbasetable")); + assertFalse(hAdmin.tableExists("hbase_table")); } finally { hAdmin.close(); } @@ -152,8 +147,8 @@ public void testCreateHBaseTable() throws Exception { @Test public void testCreateNotExistsExternalHBaseTable() throws Exception { String sql = String.format( - "CREATE EXTERNAL TABLE testcreatenotexistsexternalhbasetable (col1 text, col2 text, col3 text, col4 text) " + - "USING hbase WITH ('table'='testcreatenotexistsexternalhbasetable', 'columns'=':key,col2:a,col3:,col2:b') " + + "CREATE EXTERNAL TABLE external_hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) " + + "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:a,col3:,col2:b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); try { executeString(sql).close(); @@ -166,8 +161,8 @@ public void testCreateNotExistsExternalHBaseTable() throws Exception { @Test public void testCreateRowFieldWithNonText() throws Exception { try { - executeString("CREATE TABLE testcreaterowfieldwithnontext (rk1 int4, rk2 text, col3 text, col4 text) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='testcreaterowfieldwithnontext', 'columns'='0:key#b,1:key,col3:,col2:b', " + + executeString("CREATE TABLE hbase_mapped_table2 (rk1 int4, rk2 text, col3 text, col4 text) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'='0:key#b,1:key,col3:,col2:b', " + "'hbase.rowkey.delimiter'='_')").close(); fail("Key field type should be TEXT type"); } catch (Exception e) { @@ -177,27 +172,27 @@ public void testCreateRowFieldWithNonText() throws Exception { @Test public void testCreateExternalHBaseTable() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testcreateexternalhbasetable")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table_not_purge")); hTableDesc.addFamily(new HColumnDescriptor("col1")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE testcreateexternalhbasetable (rk text, col1 text, col2 text, col3 text) " + - "USING hbase WITH ('table'='testcreateexternalhbasetable', 'columns'=':key,col1:a,col2:,col3:b') " + + "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " + + "USING hbase WITH ('table'='external_hbase_table_not_purge', 'columns'=':key,col1:a,col2:,col3:b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("testcreateexternalhbasetable"); + assertTableExists("external_hbase_mapped_table"); - executeString("DROP TABLE testcreateexternalhbasetable").close(); + executeString("DROP TABLE external_hbase_mapped_table").close(); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); try { - assertTrue(hAdmin.tableExists("testcreateexternalhbasetable")); - hAdmin.disableTable("testcreateexternalhbasetable"); - hAdmin.deleteTable("testcreateexternalhbasetable"); + assertTrue(hAdmin.tableExists("external_hbase_table_not_purge")); + hAdmin.disableTable("external_hbase_table_not_purge"); + hAdmin.deleteTable("external_hbase_table_not_purge"); } finally { hAdmin.close(); } @@ -205,23 +200,23 @@ public void testCreateExternalHBaseTable() throws Exception { @Test public void testSimpleSelectQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testsimpleselectquery")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); hTableDesc.addFamily(new HColumnDescriptor("col1")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE testsimpleselectquery (rk text, col1 text, col2 text, col3 text) " + - "USING hbase WITH ('table'='testsimpleselectquery', 'columns'=':key,col1:a,col2:,col3:b') " + + "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " + + "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col1:a,col2:,col3:b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("testsimpleselectquery"); + assertTableExists("external_hbase_mapped_table"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("testsimpleselectquery"); + HTableInterface htable = hconn.getTable("external_hbase_table"); try { for (int i = 0; i < 100; i++) { @@ -234,34 +229,34 @@ public void testSimpleSelectQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from testsimpleselectquery where rk > '20'"); + ResultSet res = executeString("select * from external_hbase_mapped_table where rk > '20'"); assertResultSet(res); cleanupQuery(res); } finally { - executeString("DROP TABLE testsimpleselectquery PURGE").close(); + executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); htable.close(); } } @Test public void testBinaryMappedQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testbinarymappedquery")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); hTableDesc.addFamily(new HColumnDescriptor("col1")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE testbinarymappedquery (rk int8, col1 text, col2 text, col3 int4)\n " + - "USING hbase WITH ('table'='testbinarymappedquery', 'columns'=':key#b,col1:a,col2:,col3:b#b') " + + "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk int8, col1 text, col2 text, col3 int4)\n " + + "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key#b,col1:a,col2:,col3:b#b') " + "LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("testbinarymappedquery"); + assertTableExists("external_hbase_mapped_table"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("testbinarymappedquery"); + HTableInterface htable = hconn.getTable("external_hbase_table"); try { for (int i = 0; i < 100; i++) { @@ -274,12 +269,12 @@ public void testBinaryMappedQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from testbinarymappedquery where rk > 20"); + ResultSet res = executeString("select * from external_hbase_mapped_table where rk > 20"); assertResultSet(res); res.close(); //Projection - res = executeString("select col3, col2, rk from testbinarymappedquery where rk > 95"); + res = executeString("select col3, col2, rk from external_hbase_mapped_table where rk > 95"); String expected = "col3,col2,rk\n" + "-------------------------------\n" + @@ -292,29 +287,29 @@ public void testBinaryMappedQuery() throws Exception { res.close(); } finally { - executeString("DROP TABLE testbinarymappedquery PURGE").close(); + executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); htable.close(); } } @Test public void testColumnKeyValueSelectQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testcolumnkeyvalueselectquery")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); hTableDesc.addFamily(new HColumnDescriptor("col2")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE testcolumnkeyvalueselectquery (rk1 text, col2_key text, col2_value text, col3 text) " + - "USING hbase WITH ('table'='testcolumnkeyvalueselectquery', 'columns'=':key,col2:key:,col2:value:,col3:', " + + "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, col2_key text, col2_value text, col3 text) " + + "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " + "'hbase.rowkey.delimiter'='_') LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("testcolumnkeyvalueselectquery"); + assertTableExists("external_hbase_mapped_table"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("testcolumnkeyvalueselectquery"); + HTableInterface htable = hconn.getTable("external_hbase_table"); try { for (int i = 0; i < 10; i++) { @@ -326,32 +321,32 @@ public void testColumnKeyValueSelectQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from testcolumnkeyvalueselectquery where rk1 >= 'rk-0'"); + ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 >= 'rk-0'"); assertResultSet(res); cleanupQuery(res); } finally { - executeString("DROP TABLE testcolumnkeyvalueselectquery PURGE").close(); + executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); htable.close(); } } @Test public void testRowFieldSelectQuery() throws Exception { - HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("testrowfieldselectquery")); + HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table")); hTableDesc.addFamily(new HColumnDescriptor("col3")); testingCluster.getHBaseUtil().createTable(hTableDesc); String sql = String.format( - "CREATE EXTERNAL TABLE testrowfieldselectquery (rk1 text, rk2 text, col3 text) " + - "USING hbase WITH ('table'='testrowfieldselectquery', 'columns'='0:key,1:key,col3:a', " + + "CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, rk2 text, col3 text) " + + "USING hbase WITH ('table'='external_hbase_table', 'columns'='0:key,1:key,col3:a', " + "'hbase.rowkey.delimiter'='_') LOCATION '%s/external_hbase_table'", tableSpaceUri); executeString(sql).close(); - assertTableExists("testrowfieldselectquery"); + assertTableExists("external_hbase_mapped_table"); HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get(); HConnection hconn = space.getConnection(); - HTableInterface htable = hconn.getTable("testrowfieldselectquery"); + HTableInterface htable = hconn.getTable("external_hbase_table"); try { for (int i = 0; i < 100; i++) { @@ -360,11 +355,11 @@ public void testRowFieldSelectQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from testrowfieldselectquery where rk1 > 'field1-20'"); + ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 > 'field1-20'"); assertResultSet(res); cleanupQuery(res); } finally { - executeString("DROP TABLE testrowfieldselectquery PURGE").close(); + executeString("DROP TABLE external_hbase_mapped_table PURGE").close(); htable.close(); } } @@ -372,17 +367,17 @@ public void testRowFieldSelectQuery() throws Exception { @Test public void testIndexPredication() throws Exception { String sql = - "CREATE TABLE testindexpredication (rk text, col1 text, col2 text, col3 text) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='testindexpredication', 'columns'=':key,col1:a,col2:,col3:b', " + + "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b', " + "'hbase.split.rowkeys'='010,040,060,080') "; executeString(sql).close(); - assertTableExists("testindexpredication"); + assertTableExists("hbase_mapped_table"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); - hAdmin.tableExists("testindexpredication"); + hAdmin.tableExists("hbase_table"); - HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testindexpredication"); + HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); try { org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -397,13 +392,13 @@ public void testIndexPredication() throws Exception { put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes()); htable.put(put); } - assertIndexPredication(false, "testindexpredication"); + assertIndexPredication(false); - ResultSet res = executeString("select * from testindexpredication where rk >= '020' and rk <= '055'"); + ResultSet res = executeString("select * from hbase_mapped_table where rk >= '020' and rk <= '055'"); assertResultSet(res); res.close(); - res = executeString("select * from testindexpredication where rk = '021'"); + res = executeString("select * from hbase_mapped_table where rk = '021'"); String expected = "rk,col1,col2,col3\n" + "-------------------------------\n" + "021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n"; @@ -411,7 +406,7 @@ public void testIndexPredication() throws Exception { assertEquals(expected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testindexpredication PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); htable.close(); hAdmin.close(); } @@ -420,16 +415,16 @@ public void testIndexPredication() throws Exception { @Test public void testCompositeRowIndexPredication() throws Exception { - executeString("CREATE TABLE testcompositerowindexpredication (rk text, rk2 text, col1 text, col2 text, col3 text) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='testcompositerowindexpredication', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + + executeString("CREATE TABLE hbase_mapped_table (rk text, rk2 text, col1 text, col2 text, col3 text) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + "'hbase.split.rowkeys'='010,040,060,080', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("testcompositerowindexpredication"); + assertTableExists("hbase_mapped_table"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); - hAdmin.tableExists("testcompositerowindexpredication"); + hAdmin.tableExists("hbase_table"); - HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testcompositerowindexpredication"); + HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); try { org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -457,9 +452,9 @@ public void testCompositeRowIndexPredication() throws Exception { assertEquals("021_021", new String(result.getRow())); scanner.close(); - assertIndexPredication(true, "testcompositerowindexpredication"); + assertIndexPredication(true); - ResultSet res = executeString("select * from testcompositerowindexpredication where rk = '021'"); + ResultSet res = executeString("select * from hbase_mapped_table where rk = '021'"); String expected = "rk,rk2,col1,col2,col3\n" + "-------------------------------\n" + "021,021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n"; @@ -467,15 +462,15 @@ public void testCompositeRowIndexPredication() throws Exception { assertEquals(expected, resultSetToString(res)); res.close(); } finally { - executeString("DROP TABLE testcompositerowindexpredication PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); htable.close(); hAdmin.close(); } } - private void assertIndexPredication(boolean isCompositeRowKey, String table) throws Exception { + private void assertIndexPredication(boolean isCompositeRowKey) throws Exception { String postFix = isCompositeRowKey ? "_" + new String(new char[]{Character.MAX_VALUE}) : ""; - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), table); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); ScanNode scanNode = new ScanNode(1); @@ -484,7 +479,7 @@ private void assertIndexPredication(boolean isCompositeRowKey, String table) thr new ConstEval(new TextDatum("021"))); scanNode.setQual(evalNodeEq); Tablespace tablespace = TablespaceManager.getByName("cluster1").get(); - List fragments = tablespace.getSplits(table, tableDesc, scanNode); + List fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); assertEquals(1, fragments.size()); assertEquals("021", new String(((HBaseFragment)fragments.get(0)).getStartRow())); assertEquals("021" + postFix, new String(((HBaseFragment)fragments.get(0)).getStopRow())); @@ -497,7 +492,7 @@ private void assertIndexPredication(boolean isCompositeRowKey, String table) thr EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2); scanNode.setQual(evalNodeA); - fragments = tablespace.getSplits(table, tableDesc, scanNode); + fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); assertEquals(2, fragments.size()); HBaseFragment fragment1 = (HBaseFragment) fragments.get(0); assertEquals("020", new String(fragment1.getStartRow())); @@ -512,7 +507,7 @@ private void assertIndexPredication(boolean isCompositeRowKey, String table) thr new ConstEval(new TextDatum("075"))); EvalNode evalNodeB = new BinaryEval(EvalType.OR, evalNodeA, evalNode3); scanNode.setQual(evalNodeB); - fragments = tablespace.getSplits(table, tableDesc, scanNode); + fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); assertEquals(3, fragments.size()); fragment1 = (HBaseFragment) fragments.get(0); assertEquals("020", new String(fragment1.getStartRow())); @@ -535,7 +530,7 @@ private void assertIndexPredication(boolean isCompositeRowKey, String table) thr EvalNode evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5); EvalNode evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC); scanNode.setQual(evalNodeD); - fragments = tablespace.getSplits(table, tableDesc, scanNode); + fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); assertEquals(3, fragments.size()); fragment1 = (HBaseFragment) fragments.get(0); @@ -558,7 +553,7 @@ private void assertIndexPredication(boolean isCompositeRowKey, String table) thr evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5); evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC); scanNode.setQual(evalNodeD); - fragments = tablespace.getSplits(table, tableDesc, scanNode); + fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode); assertEquals(2, fragments.size()); fragment1 = (HBaseFragment) fragments.get(0); @@ -572,16 +567,16 @@ private void assertIndexPredication(boolean isCompositeRowKey, String table) thr @Test public void testNonForwardQuery() throws Exception { - executeString("CREATE TABLE testnonforwardquery (rk text, col1 text, col2 text, col3 int) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='testnonforwardquery', 'columns'=':key,col1:a,col2:,col3:#b', " + + executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:#b', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("testnonforwardquery"); + assertTableExists("hbase_mapped_table"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); HTable htable = null; try { - hAdmin.tableExists("testnonforwardquery"); - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testnonforwardquery"); + hAdmin.tableExists("hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -596,11 +591,11 @@ public void testNonForwardQuery() throws Exception { htable.put(put); } - ResultSet res = executeString("select * from testnonforwardquery"); + ResultSet res = executeString("select * from hbase_mapped_table"); assertResultSet(res); res.close(); } finally { - executeString("DROP TABLE testnonforwardquery PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); hAdmin.close(); if (htable == null) { htable.close(); @@ -610,16 +605,16 @@ public void testNonForwardQuery() throws Exception { @Test public void testJoin() throws Exception { - executeString("CREATE TABLE testjoin (rk text, col1 text, col2 text, col3 int8) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='testjoin', 'columns'=':key,col1:a,col2:,col3:b#b', " + + executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int8) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("testjoin"); + assertTableExists("hbase_mapped_table"); HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); HTable htable = null; try { - hAdmin.tableExists("testjoin"); - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testjoin"); + hAdmin.tableExists("hbase_table"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); org.apache.hadoop.hbase.util.Pair keys = htable.getStartEndKeys(); assertEquals(5, keys.getFirst().length); @@ -635,12 +630,12 @@ public void testJoin() throws Exception { } ResultSet res = executeString("select a.rk, a.col1, a.col2, a.col3, b.l_orderkey, b.l_linestatus " + - "from testjoin a " + + "from hbase_mapped_table a " + "join default.lineitem b on a.col3 = b.l_orderkey"); assertResultSet(res); res.close(); } finally { - executeString("DROP TABLE testjoin PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); hAdmin.close(); if (htable != null) { htable.close(); @@ -650,19 +645,19 @@ public void testJoin() throws Exception { @Test public void testInsertInto() throws Exception { - executeString("CREATE TABLE testinsertinto (rk text, col1 text, col2 text, col3 int4) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='testinsertinto', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); + executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); - assertTableExists("testinsertinto"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertinto"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); - executeString("insert into testinsertinto " + + executeString("insert into hbase_mapped_table " + "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertinto"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -676,7 +671,7 @@ public void testInsertInto() throws Exception { new boolean[]{false, false, false, true}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE testinsertinto PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -690,21 +685,21 @@ public void testInsertInto() throws Exception { @Test public void testInsertValues1() throws Exception { - executeString("CREATE TABLE testinsertvalues1 (rk text, col1 text, col2 text, col3 int4) " + - "TABLESPACE cluster1 USING hbase WITH ('table'='testinsertvalues1', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); + executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " + + "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); - assertTableExists("testinsertvalues1"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertvalues1"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); - executeString("insert into testinsertvalues1 select 'aaa', 'a12', 'a34', 1").close(); - executeString("insert into testinsertvalues1 select 'bbb', 'b12', 'b34', 2").close(); - executeString("insert into testinsertvalues1 select 'ccc', 'c12', 'c34', 3").close(); - executeString("insert into testinsertvalues1 select 'ddd', 'd12', 'd34', 4").close(); + executeString("insert into hbase_mapped_table select 'aaa', 'a12', 'a34', 1").close(); + executeString("insert into hbase_mapped_table select 'bbb', 'b12', 'b34', 2").close(); + executeString("insert into hbase_mapped_table select 'ccc', 'c12', 'c34', 3").close(); + executeString("insert into hbase_mapped_table select 'ddd', 'd12', 'd34', 4").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertvalues1"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -718,7 +713,7 @@ public void testInsertValues1() throws Exception { new boolean[]{false, false, false, true}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE testinsertvalues1 PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -732,12 +727,12 @@ public void testInsertValues1() throws Exception { @Test public void testInsertIntoMultiRegion() throws Exception { - executeString("CREATE TABLE testinsertintomultiregion (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintomultiregion', 'columns'=':key,col1:a', " + + executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("testinsertintomultiregion"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregion"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -752,16 +747,16 @@ public void testInsertIntoMultiRegion() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregion", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into testinsertintomultiregion " + - "select id, name from base_testinsertintomultiregion ").close(); + executeString("insert into hbase_mapped_table " + + "select id, name from base_table ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregion"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -773,8 +768,8 @@ public void testInsertIntoMultiRegion() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_testinsertintomultiregion PURGE").close(); - executeString("DROP TABLE testinsertintomultiregion PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -788,12 +783,12 @@ public void testInsertIntoMultiRegion() throws Exception { @Test public void testInsertIntoMultiRegion2() throws Exception { - executeString("CREATE TABLE testinsertintomultiregion2 (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintomultiregion2', 'columns'=':key,col1:a', " + + executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close(); - assertTableExists("testinsertintomultiregion2"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregion2"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -807,16 +802,16 @@ public void testInsertIntoMultiRegion2() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(i + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregion2", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into testinsertintomultiregion2 " + - "select id, name from base_testinsertintomultiregion2 ").close(); + executeString("insert into hbase_mapped_table " + + "select id, name from base_table ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregion2"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -828,8 +823,8 @@ public void testInsertIntoMultiRegion2() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_testinsertintomultiregion2 PURGE").close(); - executeString("DROP TABLE testinsertintomultiregion2 PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -846,12 +841,12 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { String splitFilePath = currentDatasetPath + "/splits.data"; executeString( - "CREATE TABLE testinsertintomultiregionwithsplitfile (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintomultiregionwithsplitfile', 'columns'=':key,col1:a', " + + "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys.file'='" + splitFilePath + "')").close(); - assertTableExists("testinsertintomultiregionwithsplitfile"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregionwithsplitfile"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -866,16 +861,16 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregionwithsplitfile", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into testinsertintomultiregionwithsplitfile " + - "select id, name from base_testinsertintomultiregionwithsplitfile ").close(); + executeString("insert into hbase_mapped_table " + + "select id, name from base_table ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregionwithsplitfile"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -887,8 +882,8 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_testinsertintomultiregionwithsplitfile PURGE").close(); - executeString("DROP TABLE testinsertintomultiregionwithsplitfile PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -903,13 +898,13 @@ public void testInsertIntoMultiRegionWithSplitFile() throws Exception { @Test public void testInsertIntoMultiRegionMultiRowFields() throws Exception { executeString( - "CREATE TABLE testinsertintomultiregionmultirowfields (rk1 text, rk2 text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintomultiregionmultirowfields', 'columns'='0:key,1:key,col1:a', " + + "CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a', " + "'hbase.split.rowkeys'='001,002,003,004,005,006,007,008,009', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("testinsertintomultiregionmultirowfields"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintomultiregionmultirowfields"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -925,16 +920,16 @@ public void testInsertIntoMultiRegionMultiRowFields() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|" + (i + 100) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintomultiregionmultirowfields", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into testinsertintomultiregionmultirowfields " + - "select id1, id2, name from base_testinsertintomultiregionmultirowfields ").close(); + executeString("insert into hbase_mapped_table " + + "select id1, id2, name from base_table ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintomultiregionmultirowfields"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -946,8 +941,8 @@ public void testInsertIntoMultiRegionMultiRowFields() throws Exception { new boolean[]{false, false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_testinsertintomultiregionmultirowfields PURGE").close(); - executeString("DROP TABLE testinsertintomultiregionmultirowfields PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -961,12 +956,12 @@ public void testInsertIntoMultiRegionMultiRowFields() throws Exception { @Test public void testInsertIntoBinaryMultiRegion() throws Exception { - executeString("CREATE TABLE testinsertintobinarymultiregion (rk int4, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintobinarymultiregion', 'columns'=':key#b,col1:a', " + + executeString("CREATE TABLE hbase_mapped_table (rk int4, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key#b,col1:a', " + "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close(); - assertTableExists("testinsertintobinarymultiregion"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintobinarymultiregion"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -980,16 +975,16 @@ public void testInsertIntoBinaryMultiRegion() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(i + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintobinarymultiregion", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into testinsertintobinarymultiregion " + - "select id, name from base_testinsertintobinarymultiregion ").close(); + executeString("insert into hbase_mapped_table " + + "select id, name from base_table ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintobinarymultiregion"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -1001,8 +996,8 @@ public void testInsertIntoBinaryMultiRegion() throws Exception { new boolean[]{true, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_testinsertintobinarymultiregion PURGE").close(); - executeString("DROP TABLE testinsertintobinarymultiregion PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -1017,12 +1012,12 @@ public void testInsertIntoBinaryMultiRegion() throws Exception { @Test public void testInsertIntoColumnKeyValue() throws Exception { executeString( - "CREATE TABLE testinsertintocolumnkeyvalue (rk text, col2_key text, col2_value text, col3 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintocolumnkeyvalue', 'columns'=':key,col2:key:,col2:value:,col3:', " + + "CREATE TABLE hbase_mapped_table (rk text, col2_key text, col2_value text, col3 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("testinsertintocolumnkeyvalue"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintocolumnkeyvalue"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -1040,16 +1035,16 @@ public void testInsertIntoColumnKeyValue() throws Exception { datas.add(i + "|ck-" + j + "|value-" + j + "|col3-" + i); } } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintocolumnkeyvalue", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); - executeString("insert into testinsertintocolumnkeyvalue " + - "select rk, col2_key, col2_value, col3 from base_testinsertintocolumnkeyvalue ").close(); + executeString("insert into hbase_mapped_table " + + "select rk, col2_key, col2_value, col3 from base_table ").close(); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintocolumnkeyvalue"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col2")); @@ -1061,7 +1056,7 @@ public void testInsertIntoColumnKeyValue() throws Exception { new byte[][]{null, null, null}, new boolean[]{false, false, false}, tableDesc.getSchema())); - ResultSet res = executeString("select * from testinsertintocolumnkeyvalue"); + ResultSet res = executeString("select * from hbase_mapped_table"); String expected = "rk,col2_key,col2_value,col3\n" + "-------------------------------\n" + @@ -1091,8 +1086,8 @@ public void testInsertIntoColumnKeyValue() throws Exception { res.close(); } finally { - executeString("DROP TABLE base_testinsertintocolumnkeyvalue PURGE").close(); - executeString("DROP TABLE testinsertintocolumnkeyvalue PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -1107,11 +1102,11 @@ public void testInsertIntoColumnKeyValue() throws Exception { @Test public void testInsertIntoDifferentType() throws Exception { executeString( - "CREATE TABLE testinsertintodifferenttype (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintodifferenttype', 'columns'=':key,col1:a', " + + "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close(); - assertTableExists("testinsertintodifferenttype"); + assertTableExists("hbase_mapped_table"); // create test table KeyValueSet tableOptions = new KeyValueSet(); @@ -1125,39 +1120,39 @@ public void testInsertIntoDifferentType() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(i + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintodifferenttype", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); try { - executeString("insert into testinsertintodifferenttype " + - "select id, name from base_testinsertintodifferenttype ").close(); + executeString("insert into hbase_mapped_table " + + "select id, name from base_table ").close(); fail("If inserting data type different with target table data type, should throw exception"); } catch (Exception e) { assertTrue(e.getMessage().indexOf("is different column type with") >= 0); } finally { - executeString("DROP TABLE base_testinsertintodifferenttype PURGE").close(); - executeString("DROP TABLE testinsertintodifferenttype PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); } } @Test public void testInsertIntoRowField() throws Exception { executeString( - "CREATE TABLE testinsertintorowfield (rk1 text, rk2 text, col1 text, col2 text, col3 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintorowfield', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + + "CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text, col2 text, col3 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " + "'hbase.rowkey.delimiter'='_')").close(); - assertTableExists("testinsertintorowfield"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintorowfield"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); - executeString("insert into testinsertintorowfield " + + executeString("insert into hbase_mapped_table " + "select l_orderkey::text, l_partkey::text, l_shipdate, l_returnflag, l_suppkey::text from default.lineitem "); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintorowfield"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -1171,7 +1166,7 @@ public void testInsertIntoRowField() throws Exception { new boolean[]{false, false, false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE testinsertintorowfield PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -1198,23 +1193,23 @@ public void testCTAS() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testctas", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); executeString( - "CREATE TABLE testctas (rk text, col1 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testctas', 'columns'=':key,col1:a', " + + "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " + "'hbase.split.rowkeys'='010,040,060,080') as" + - " select id, name from base_testctas" + " select id, name from base_table" ).close(); - assertTableExists("testctas"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testctas"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); HTable htable = null; ResultScanner scanner = null; try { - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testctas"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -1226,8 +1221,8 @@ public void testCTAS() throws Exception { new boolean[]{false, false}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE base_testctas PURGE").close(); - executeString("DROP TABLE testctas PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); if (scanner != null) { scanner.close(); @@ -1239,9 +1234,9 @@ public void testCTAS() throws Exception { // TODO - rollback should support its corresponding hbase table HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf()); - if (hAdmin.tableExists("testctas")) { - hAdmin.disableTable("testctas"); - hAdmin.deleteTable("testctas"); + if (hAdmin.tableExists("hbase_table")) { + hAdmin.disableTable("hbase_table"); + hAdmin.deleteTable("hbase_table"); } } } @@ -1249,11 +1244,11 @@ public void testCTAS() throws Exception { @Test public void testInsertIntoUsingPut() throws Exception { executeString( - "CREATE TABLE testinsertintousingput (rk text, col1 text, col2 text, col3 int4) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintousingput', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); + "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close(); - assertTableExists("testinsertintousingput"); - TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "testinsertintousingput"); + assertTableExists("hbase_mapped_table"); + TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table"); Map sessions = new HashMap(); sessions.put(HBaseStorageConstants.INSERT_PUT_MODE, "true"); @@ -1263,11 +1258,11 @@ public void testInsertIntoUsingPut() throws Exception { ResultScanner scanner = null; try { executeString( - "insert into testinsertintousingput " + + "insert into hbase_mapped_table " + "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem" ).close(); - htable = new HTable(testingCluster.getHBaseUtil().getConf(), "testinsertintousingput"); + htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table"); Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("col1")); @@ -1282,7 +1277,7 @@ public void testInsertIntoUsingPut() throws Exception { new boolean[]{false, false, false, true}, tableDesc.getSchema())); } finally { - executeString("DROP TABLE testinsertintousingput PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); client.unsetSessionVariables(TUtil.newList(HBaseStorageConstants.INSERT_PUT_MODE)); @@ -1299,11 +1294,11 @@ public void testInsertIntoUsingPut() throws Exception { @Test public void testInsertIntoLocation() throws Exception { executeString( - "CREATE TABLE testinsertintolocation (rk text, col1 text, col2 text) TABLESPACE cluster1 " + - "USING hbase WITH ('table'='testinsertintolocation', 'columns'=':key,col1:a,col2:', " + + "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text) TABLESPACE cluster1 " + + "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:', " + "'hbase.split.rowkeys'='010,040,060,080')").close(); - assertTableExists("testinsertintolocation"); + assertTableExists("hbase_mapped_table"); try { // create test table @@ -1320,11 +1315,11 @@ public void testInsertIntoLocation() throws Exception { for (int i = 99; i >= 0; i--) { datas.add(df.format(i) + "|value" + i + "|comment-" + i); } - TajoTestingCluster.createTable(getCurrentDatabase() + ".base_testinsertintolocation", + TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table", schema, tableOptions, datas.toArray(new String[]{}), 2); executeString("insert into location '/tmp/hfile_test' " + - "select id, name, comment from base_testinsertintolocation ").close(); + "select id, name, comment from base_table ").close(); FileSystem fs = testingCluster.getDefaultFileSystem(); Path path = new Path("/tmp/hfile_test"); @@ -1344,8 +1339,8 @@ public void testInsertIntoLocation() throws Exception { index++; } } finally { - executeString("DROP TABLE base_testinsertintolocation PURGE").close(); - executeString("DROP TABLE testinsertintolocation PURGE").close(); + executeString("DROP TABLE base_table PURGE").close(); + executeString("DROP TABLE hbase_mapped_table PURGE").close(); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java index 7987421467..265f075cb3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java +++ b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java @@ -37,7 +37,6 @@ import static junit.framework.TestCase.assertEquals; import static org.junit.Assert.assertNotEquals; -@net.jcip.annotations.NotThreadSafe public class TestHAServiceHDFSImpl { private TajoTestingCluster cluster; diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index 8a73c02aa1..9b4c06997d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -19,7 +19,6 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Lists; -import net.jcip.annotations.NotThreadSafe; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; @@ -62,7 +61,6 @@ import static org.junit.Assert.*; -@NotThreadSafe public class TestKillQuery { private static TajoTestingCluster cluster; private static TajoConf conf; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java index aa63968842..a91fc30513 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java @@ -67,7 +67,7 @@ public void tearDown(){ @Test public void testGet() throws IOException { Random rnd = new Random(); - QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; String sid = "1"; String partId = "1"; @@ -116,7 +116,7 @@ public void testAdjustFetchProcess() { @Test public void testStatus() throws Exception { Random rnd = new Random(); - QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; String sid = "1"; String ta = "1_0"; String partId = "1"; @@ -145,7 +145,7 @@ public void testStatus() throws Exception { @Test public void testNoContentFetch() throws Exception { - QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; String sid = "1"; String ta = "1_0"; String partId = "1"; @@ -176,7 +176,7 @@ public void testNoContentFetch() throws Exception { public void testFailureStatus() throws Exception { Random rnd = new Random(); - QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; String sid = "1"; String ta = "1_0"; String partId = "1"; @@ -208,7 +208,7 @@ public void testFailureStatus() throws Exception { @Test public void testServerFailure() throws Exception { - QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); + QueryId queryId = QueryIdFactory.NULL_QUERY_ID; String sid = "1"; String ta = "1_0"; String partId = "1"; diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql index fd45f920a7..208445628b 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql @@ -1 +1 @@ -SELECT "aGe", "teXt", "Number" FROM "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS"; \ No newline at end of file +SELECT "aGe", "teXt", "Number" FROM "TABLE1"; \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql index a5a28c180b..4b05657653 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql @@ -1 +1 @@ -SELECT "tb1"."aGe" as "AgE", "tb1"."teXt" as "TEXT", "Number" as "NUMBER" FROM "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS" "tb1"; \ No newline at end of file +SELECT "tb1"."aGe" as "AgE", "tb1"."teXt" as "TEXT", "Number" as "NUMBER" FROM "TABLE1" "tb1"; \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql index 9e86cb70f7..fbb00a02c4 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql @@ -6,7 +6,7 @@ FROM ( "tb1"."teXt", "Number" FROM - "TESTDELIMITEDIDENTIFIERWITHMIXEDCHARACTERS" "tb1" + "TABLE1" "tb1" UNION @@ -15,7 +15,7 @@ FROM ( "teXt", "Number" FROM - "testDelimitedIdentifierWithMixedCharacters" + "tablE1" ) T1 ORDER BY "aGe"; diff --git a/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql b/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql index 22d2803bcd..f794d21470 100644 --- a/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql +++ b/tajo-core/src/test/resources/queries/TestCreateTable/testNestedRecord2.sql @@ -1 +1 @@ -CREATE TABLE D10.nested_table2 (f1 int, nf1 record (f1 int4, f3 double), nf2 record (f1 int4, nf1 record (f1 int4, f2 text), nf2 record (f1 int4, f2 text), f2 double), f2 text); \ No newline at end of file +CREATE TABLE D9.nested_table2 (f1 int, nf1 record (f1 int4, f3 double), nf2 record (f1 int4, nf1 record (f1 int4, f2 text), nf2 record (f1 int4, f2 text), f2 double), f2 text); \ No newline at end of file From 97037fc4680c936c2a76f3fc00417880b16d63bc Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sun, 12 Jul 2015 21:46:32 +0900 Subject: [PATCH 51/80] Revert "change parallel fork to parallel thread" This reverts commit 9a958082fb578396d4f99acd9359f4016e6bfe68. --- tajo-core/pom.xml | 5 +- .../org/apache/tajo/TajoTestingCluster.java | 101 +++++++++--------- .../tajo/engine/query/TestGroupByQuery.java | 15 ++- .../tajo/engine/query/TestInnerJoinQuery.java | 1 - .../tajo/engine/query/TestInsertQuery.java | 1 - .../testDistinctAggregationCaseByCase3.sql | 2 +- 6 files changed, 58 insertions(+), 67 deletions(-) diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 6e4a33fb55..6ad6956730 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -150,6 +150,7 @@ org.codehaus.mojo exec-maven-plugin + 1.2 generate-sources @@ -572,7 +573,6 @@ org.apache.maven.plugins maven-surefire-plugin - 2.18.1 test @@ -584,9 +584,6 @@ ${maven.fork.count} true - classesAndMethods - 1 - ${maven.fork.count} false -Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 true diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 70e3a208e2..8cb9e29d42 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -47,8 +47,10 @@ import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.storage.FileTablespace; import org.apache.tajo.storage.TablespaceManager; -import org.apache.tajo.util.*; -import org.apache.tajo.util.FileUtil; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.KeyValueSet; +import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.Pair; import org.apache.tajo.worker.TajoWorker; import java.io.File; @@ -69,7 +71,6 @@ public class TajoTestingCluster { private MiniDFSCluster dfsCluster; private MiniCatalogServer catalogServer; private HBaseTestClusterUtil hbaseUtil; - private TajoClient tajoClient; private TajoMaster tajoMaster; private List tajoWorkers = new ArrayList(); @@ -126,9 +127,8 @@ void initPropertiesAndConfigs() { conf.setClassVar(ConfVars.GLOBAL_PLAN_REWRITE_RULE_PROVIDER_CLASS, GlobalPlanTestRuleProvider.class); conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES.varname, 4); - conf.setInt(ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.varname, 150); - conf.setInt(ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY.varname, 100); - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 4); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 2000); + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 3);; conf.setInt(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM.varname, 2); // Client API RPC @@ -455,8 +455,6 @@ public void shutdownMiniTajoCluster() { eachWorker.stopWorkerForce(); } tajoWorkers.clear(); - - FileUtil.cleanup(null, this.tajoClient); this.tajoMaster= null; } @@ -620,19 +618,15 @@ public static ResultSet run(String[] names, } } - public TajoClient getTajoClient() throws InterruptedException, IOException { - if(tajoClient == null) { - while(true) { - if(getMaster().isMasterRunning()) { - break; - } - Thread.sleep(1000); + public static TajoClient newTajoClient(TajoTestingCluster util) throws InterruptedException, IOException { + while(true) { + if(util.getMaster().isMasterRunning()) { + break; } - - this.tajoClient = new TajoClientImpl(ServiceTrackerFactory.get(getConfiguration())); + Thread.sleep(1000); } - - return tajoClient; + TajoConf conf = util.getConfiguration(); + return new TajoClientImpl(ServiceTrackerFactory.get(conf)); } public static void createTable(String tableName, Schema schema, @@ -644,44 +638,47 @@ public static void createTable(String tableName, Schema schema, KeyValueSet tableOption, String[] tableDatas, int numDataFiles) throws Exception { TpchTestBase instance = TpchTestBase.getInstance(); TajoTestingCluster util = instance.getTestingCluster(); - TajoClient client = util.getTajoClient(); - FileSystem fs = util.getDefaultFileSystem(); - Path rootDir = TajoConf.getWarehouseDir(util.getConfiguration()); - - if (!fs.exists(rootDir)) { - fs.mkdirs(rootDir); - } - Path tablePath; - if (CatalogUtil.isFQTableName(tableName)) { - Pair name = CatalogUtil.separateQualifierAndName(tableName); - tablePath = new Path(rootDir, new Path(name.getFirst(), name.getSecond())); - } else { - tablePath = new Path(rootDir, tableName); - } - - fs.mkdirs(tablePath); - if (tableDatas.length > 0) { - int recordPerFile = tableDatas.length / numDataFiles; - if (recordPerFile == 0) { - recordPerFile = 1; + TajoClient client = newTajoClient(util); + try { + FileSystem fs = util.getDefaultFileSystem(); + Path rootDir = TajoConf.getWarehouseDir(util.getConfiguration()); + if (!fs.exists(rootDir)) { + fs.mkdirs(rootDir); + } + Path tablePath; + if (CatalogUtil.isFQTableName(tableName)) { + Pair name = CatalogUtil.separateQualifierAndName(tableName); + tablePath = new Path(rootDir, new Path(name.getFirst(), name.getSecond())); + } else { + tablePath = new Path(rootDir, tableName); } - FSDataOutputStream out = null; - for (int j = 0; j < tableDatas.length; j++) { - if (out == null || j % recordPerFile == 0) { - if (out != null) { - out.close(); + + fs.mkdirs(tablePath); + if (tableDatas.length > 0) { + int recordPerFile = tableDatas.length / numDataFiles; + if (recordPerFile == 0) { + recordPerFile = 1; + } + FSDataOutputStream out = null; + for (int j = 0; j < tableDatas.length; j++) { + if (out == null || j % recordPerFile == 0) { + if (out != null) { + out.close(); + } + Path dfsPath = new Path(tablePath, tableName + j + ".tbl"); + out = fs.create(dfsPath); } - Path dfsPath = new Path(tablePath, tableName + j + ".tbl"); - out = fs.create(dfsPath); + out.write((tableDatas[j] + "\n").getBytes()); + } + if (out != null) { + out.close(); } - out.write((tableDatas[j] + "\n").getBytes()); - } - if (out != null) { - out.close(); } + TableMeta meta = CatalogUtil.newTableMeta("CSV", tableOption); + client.createExternalTable(tableName, schema, tablePath.toUri(), meta); + } finally { + client.close(); } - TableMeta meta = CatalogUtil.newTableMeta("CSV", tableOption); - client.createExternalTable(tableName, schema, tablePath.toUri(), meta); } /** diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java index 345ceaee7a..305536281d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java @@ -46,7 +46,6 @@ @Category(IntegrationTest.class) @RunWith(Parameterized.class) -@net.jcip.annotations.NotThreadSafe public class TestGroupByQuery extends QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(TestGroupByQuery.class); @@ -433,12 +432,12 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { schema.addColumn("qty", Type.INT4); schema.addColumn("qty2", Type.FLOAT8); String[] data = new String[]{"1|a|3|3.0", "1|a|4|4.0", "1|b|5|5.0", "2|a|1|6.0", "2|c|2|7.0", "2|d|3|8.0"}; - TajoTestingCluster.createTable("testdistinctaggregationcasebycase11", schema, tableOptions, data); + TajoTestingCluster.createTable("table10", schema, tableOptions, data); res = executeString("select id, count(distinct code), " + "avg(qty), min(qty), max(qty), sum(qty), " + "cast(avg(qty2) as INT8), cast(min(qty2) as INT8), cast(max(qty2) as INT8), cast(sum(qty2) as INT8) " + - "from testdistinctaggregationcasebycase11 group by id"); + "from table10 group by id"); String expected = "id,?count_4,?avg_5,?min_6,?max_7,?sum_8,?cast_9,?cast_10,?cast_11,?cast_12\n" + "-------------------------------\n" + @@ -449,7 +448,7 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { // multiple distinct with expression res = executeString( - "select count(distinct code) + count(distinct qty) from testdistinctaggregationcasebycase11" + "select count(distinct code) + count(distinct qty) from table10" ); expected = "?plus_2\n" + @@ -460,7 +459,7 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { res.close(); res = executeString( - "select id, count(distinct code) + count(distinct qty) from testdistinctaggregationcasebycase11 group by id" + "select id, count(distinct code) + count(distinct qty) from table10 group by id" ); expected = "id,?plus_2\n" + @@ -471,7 +470,7 @@ public final void testDistinctAggregationCasebyCase11() throws Exception { assertEquals(expected, resultSetToString(res)); res.close(); - executeString("DROP TABLE testdistinctaggregationcasebycase11 PURGE").close(); + executeString("DROP TABLE table10 PURGE").close(); } @Test @@ -495,13 +494,13 @@ public final void testDistinctAggregationCaseByCase3() throws Exception { "a|b-3|\\N" }; - TajoTestingCluster.createTable("testdistinctaggregationcasebycase3", schema, tableOptions, data); + TajoTestingCluster.createTable("table10", schema, tableOptions, data); ResultSet res = executeQuery(); assertResultSet(res); cleanupQuery(res); - executeString("DROP TABLE testdistinctaggregationcasebycase3 PURGE").close(); + executeString("DROP TABLE table10 PURGE").close(); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java index 77473d0774..6eedb4223a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInnerJoinQuery.java @@ -33,7 +33,6 @@ @Category(IntegrationTest.class) @RunWith(Parameterized.class) @NamedTest("TestJoinQuery") -@net.jcip.annotations.NotThreadSafe public class TestInnerJoinQuery extends TestJoinQuery { public TestInnerJoinQuery(String joinOption) throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java index cb156d0e9c..7a671d8386 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java @@ -41,7 +41,6 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestInsertQuery extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql b/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql index 76b3fe36a8..925cadd21c 100644 --- a/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql +++ b/tajo-core/src/test/resources/queries/TestGroupByQuery/testDistinctAggregationCaseByCase3.sql @@ -3,6 +3,6 @@ select count(distinct col2) as cnt1, count(distinct case when col3 is not null then col2 else null end) as cnt2 from - testdistinctaggregationcasebycase3 + table10 group by col1; \ No newline at end of file From 0d891977ee099ac834a136e7456e2c88ea345c67 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sun, 12 Jul 2015 21:47:52 +0900 Subject: [PATCH 52/80] Revert "add NotThreadSafe annotation" This reverts commit 81bb329fe1b30ac885ebe822d165d9ac49d3707e. --- .../java/org/apache/tajo/engine/planner/TestLogicalPlanner.java | 1 - 1 file changed, 1 deletion(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java index 4e673e8bb8..0f377630de 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java @@ -60,7 +60,6 @@ import static org.junit.Assert.*; import static org.junit.Assert.assertEquals; -@net.jcip.annotations.NotThreadSafe public class TestLogicalPlanner { private static TajoTestingCluster util; private static CatalogService catalog; From a461bf368361d4131af08c73fca865e8b5d90519 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sun, 12 Jul 2015 21:49:49 +0900 Subject: [PATCH 53/80] Revert "add NotThreadSafe annotation" This reverts commit 18207d29c7ff3061ad91bb96b584fcfd3b7f069e. --- .travis.yml | 1 + tajo-core/pom.xml | 2 +- .../org/apache/tajo/QueryTestCaseBase.java | 18 +++++-- .../org/apache/tajo/benchmark/TestTPCH.java | 3 +- .../apache/tajo/cli/tools/TestTajoDump.java | 1 - .../apache/tajo/client/TestTajoClient.java | 48 +++++++++++-------- .../engine/function/TestBuiltinFunctions.java | 1 - .../engine/planner/TestQueryValidation.java | 1 - .../tajo/engine/query/TestAlterTable.java | 1 - .../engine/query/TestAlterTablespace.java | 1 - .../tajo/engine/query/TestCTASQuery.java | 1 - .../tajo/engine/query/TestCaseByCases.java | 1 - .../tajo/engine/query/TestCreateDatabase.java | 1 - .../tajo/engine/query/TestCreateTable.java | 1 - .../tajo/engine/query/TestDropTable.java | 1 - .../tajo/engine/query/TestHBaseTable.java | 1 - .../query/TestJoinOnPartitionedTables.java | 1 + .../tajo/engine/query/TestJoinQuery.java | 1 - .../tajo/engine/query/TestNetTypes.java | 1 - .../engine/query/TestSelectNestedRecord.java | 1 - .../tajo/engine/query/TestSelectQuery.java | 1 - .../engine/query/TestSetSessionQuery.java | 6 ++- .../tajo/engine/query/TestSortQuery.java | 3 +- .../engine/query/TestTablePartitions.java | 1 - .../tajo/engine/query/TestTableSubQuery.java | 1 - .../tajo/engine/query/TestTruncateTable.java | 1 - .../tajo/engine/query/TestUnionQuery.java | 1 - .../tajo/engine/query/TestWindowQuery.java | 1 - .../tajo/jdbc/TestTajoDatabaseMetaData.java | 1 - .../org/apache/tajo/jdbc/TestTajoJdbc.java | 1 - ...estNonForwardQueryResultSystemScanner.java | 43 ++++++++++++++++- .../tajo/master/rule/TestMasterRules.java | 15 +++--- .../querymaster/TestTaskStatusUpdate.java | 1 - .../util/history/TestHistoryWriterReader.java | 1 - .../tajo/worker/TestNodeResourceManager.java | 2 - .../tajo/worker/TestNodeStatusUpdater.java | 1 - .../apache/tajo/worker/TestTaskExecutor.java | 1 - .../apache/tajo/worker/TestTaskManager.java | 1 - .../ws/rs/resources/TestClusterResource.java | 17 +++---- .../rs/resources/TestDatabasesResource.java | 24 +++++----- .../rs/resources/TestFunctionsResource.java | 1 - .../ws/rs/resources/TestQueryResource.java | 1 - .../rs/resources/TestQueryResultResource.java | 1 - .../ws/rs/resources/TestSessionsResource.java | 24 +++++----- .../ws/rs/resources/TestTablesResource.java | 1 - 45 files changed, 134 insertions(+), 104 deletions(-) diff --git a/.travis.yml b/.travis.yml index dd8485674e..7eb94a3b84 100644 --- a/.travis.yml +++ b/.travis.yml @@ -35,6 +35,7 @@ notifications: irc: "chat.freenode.net#tajo" before_install: ulimit -t 514029 -n 3000 +before_install: ulimit -a install: ./dev-support/travis-install-dependencies.sh script: diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 6ad6956730..54a980475e 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -585,7 +585,7 @@ ${maven.fork.count} true false - -Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 + -Xms512m -Xmx1200m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 true true diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java index f873c3a70a..a323f252f6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java @@ -57,13 +57,22 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.lang.annotation.*; +import java.lang.annotation.Annotation; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; import java.lang.reflect.Method; import java.net.URL; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.sql.SQLException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import static org.junit.Assert.*; @@ -135,8 +144,7 @@ *
  • Each query file must have the file extension '.sql'.
  • * */ -@net.jcip.annotations.NotThreadSafe -public abstract class QueryTestCaseBase { +public class QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(QueryTestCaseBase.class); protected static final TpchTestBase testBase; protected static final TajoTestingCluster testingCluster; @@ -223,7 +231,7 @@ public void printTestName() { /* protect a travis stalled build */ System.out.println("Run: " + name.getMethodName() + " Used memory: " + ((Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) - / (1024 * 1024)) + " MBytes, Running threads:" + Thread.activeCount()); + / (1024 * 1024)) + "MBytes"); } public QueryTestCaseBase() { diff --git a/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java index d171be125d..9b7db2de0b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java +++ b/tajo-core/src/test/java/org/apache/tajo/benchmark/TestTPCH.java @@ -24,8 +24,9 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import java.sql.ResultSet; + @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestTPCH extends QueryTestCaseBase { public TestTPCH() { diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java b/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java index 8d73ad08aa..b371be29e8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java +++ b/tajo-core/src/test/java/org/apache/tajo/cli/tools/TestTajoDump.java @@ -25,7 +25,6 @@ import java.io.ByteArrayOutputStream; import java.io.PrintWriter; -@net.jcip.annotations.NotThreadSafe public class TestTajoDump extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java index 228f0be80e..73b97fa37f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java +++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java @@ -22,6 +22,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.protobuf.ServiceException; +import net.jcip.annotations.NotThreadSafe; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileStatus; @@ -41,7 +42,9 @@ import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.storage.StorageUtil; import org.apache.tajo.util.CommonTestingUtil; -import org.junit.*; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; import org.junit.experimental.categories.Category; import java.io.IOException; @@ -53,33 +56,24 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) +@NotThreadSafe public class TestTajoClient { private static TajoTestingCluster cluster; private static TajoConf conf; + private static TajoClient client; private static Path testDir; - private TajoClient client; @BeforeClass - public static void setUpClass() throws Exception { + public static void setUp() throws Exception { cluster = TpchTestBase.getInstance().getTestingCluster(); conf = cluster.getConfiguration(); + client = cluster.newTajoClient(); testDir = CommonTestingUtil.getTestDir(); } @AfterClass - public static void tearDownClass() throws Exception { - } - - @Before - public void setUp() throws Exception { - assertNull(client); - client = cluster.newTajoClient(); - } - - @After - public void tearDown() throws Exception { + public static void tearDown() throws Exception { client.close(); - client = null; } private static Path writeTmpTable(String tableName) throws IOException { @@ -90,14 +84,20 @@ private static Path writeTmpTable(String tableName) throws IOException { @Test public final void testCreateAndDropDatabases() throws ServiceException { + int currentNum = client.getAllDatabaseNames().size(); + String prefix = CatalogUtil.normalizeIdentifier("testCreateDatabase_"); for (int i = 0; i < 10; i++) { + // test allDatabaseNames + assertEquals(currentNum + i, client.getAllDatabaseNames().size()); + // test existence assertFalse(client.existDatabase(prefix + i)); assertTrue(client.createDatabase(prefix + i)); assertTrue(client.existDatabase(prefix + i)); // test allDatabaseNames + assertEquals(currentNum + i + 1, client.getAllDatabaseNames().size()); assertTrue(client.getAllDatabaseNames().contains(prefix + i)); } @@ -109,25 +109,30 @@ public final void testCreateAndDropDatabases() throws ServiceException { assertFalse(client.existDatabase(prefix + i)); assertFalse(client.getAllDatabaseNames().contains(prefix + i)); } + + assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test public final void testCurrentDatabase() throws IOException, ServiceException, InterruptedException { - + int currentNum = client.getAllDatabaseNames().size(); assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase()); String databaseName = CatalogUtil.normalizeIdentifier("testcurrentdatabase"); assertTrue(client.createDatabase(databaseName)); - + assertEquals(currentNum + 1, client.getAllDatabaseNames().size()); assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase()); assertTrue(client.selectDatabase(databaseName)); assertEquals(databaseName, client.getCurrentDatabase()); assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME)); assertTrue(client.dropDatabase(databaseName)); + + assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test public final void testSelectDatabaseToInvalidOne() throws IOException, ServiceException, InterruptedException { + int currentNum = client.getAllDatabaseNames().size(); assertFalse(client.existDatabase("invaliddatabase")); try { @@ -136,12 +141,13 @@ public final void testSelectDatabaseToInvalidOne() throws IOException, ServiceEx } catch (Throwable t) { assertFalse(false); } + + assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test public final void testDropCurrentDatabase() throws IOException, ServiceException, InterruptedException { - assertTrue(0 < client.getAllDatabaseNames().size()); - + int currentNum = client.getAllDatabaseNames().size(); String databaseName = CatalogUtil.normalizeIdentifier("testdropcurrentdatabase"); assertTrue(client.createDatabase(databaseName)); assertTrue(client.selectDatabase(databaseName)); @@ -156,6 +162,7 @@ public final void testDropCurrentDatabase() throws IOException, ServiceException assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME)); assertTrue(client.dropDatabase(databaseName)); + assertEquals(currentNum, client.getAllDatabaseNames().size()); } @Test @@ -613,7 +620,8 @@ public final void testGetFinishedQueryList() throws IOException, resultSet = client.executeQueryAndGetResult("select * from " + tableName + " order by deptname"); assertNotNull(resultSet); - assertTrue(numFinishedQueries + 2 <= client.getFinishedQueryList().size()); + assertEquals(numFinishedQueries + 2, client.getFinishedQueryList().size()); + resultSet.close(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java index 705aa69eb1..72fdd6fa06 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java @@ -34,7 +34,6 @@ import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestBuiltinFunctions extends QueryTestCaseBase { public TestBuiltinFunctions() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java index 12d227d59e..fd60a5e53f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestQueryValidation.java @@ -24,7 +24,6 @@ import java.io.IOException; -@net.jcip.annotations.NotThreadSafe public class TestQueryValidation extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java index 9a56efc1d2..44fa1f3594 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java @@ -27,7 +27,6 @@ import java.util.List; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestAlterTable extends QueryTestCaseBase { @Test public final void testAlterTableName() throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java index 32c1f49893..b0ec92d2e5 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java @@ -27,7 +27,6 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestAlterTablespace extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java index 71ff04992e..2758d7d1f9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java @@ -48,7 +48,6 @@ * Test CREATE TABLE AS SELECT statements */ @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestCTASQuery extends QueryTestCaseBase { public TestCTASQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java index a2f30eb785..bcf00f88eb 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertEquals; -@net.jcip.annotations.NotThreadSafe public class TestCaseByCases extends QueryTestCaseBase { public TestCaseByCases() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java index 78a0c264ff..453c174ca3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java @@ -27,7 +27,6 @@ import java.sql.ResultSet; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestCreateDatabase extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java index 539a4a4f16..b19a488cb4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java @@ -36,7 +36,6 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestCreateTable extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java index f57a850f26..00201568e8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestDropTable.java @@ -26,7 +26,6 @@ import java.util.List; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestDropTable extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java index 6adcc1fa56..569111c595 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java @@ -61,7 +61,6 @@ import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestHBaseTable extends QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(TestHBaseTable.class); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java index da0f59d29d..d769c13dc7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinOnPartitionedTables.java @@ -46,6 +46,7 @@ @Category(IntegrationTest.class) @RunWith(Parameterized.class) @NamedTest("TestJoinQuery") +@net.jcip.annotations.NotThreadSafe public class TestJoinOnPartitionedTables extends TestJoinQuery { public TestJoinOnPartitionedTables(String joinOption) throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java index 2b8f009daf..dd67e06496 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java @@ -51,7 +51,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -@net.jcip.annotations.NotThreadSafe public class TestJoinQuery extends QueryTestCaseBase { private static final Log LOG = LogFactory.getLog(TestJoinQuery.class); private static int reference = 0; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java index 61075cdb62..bd8f83049b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNetTypes.java @@ -24,7 +24,6 @@ import java.sql.ResultSet; -@net.jcip.annotations.NotThreadSafe public class TestNetTypes extends QueryTestCaseBase { @Before diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java index 8499b3f9b6..9f8a5fdbd7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectNestedRecord.java @@ -27,7 +27,6 @@ import static org.junit.Assert.assertEquals; -@net.jcip.annotations.NotThreadSafe public class TestSelectNestedRecord extends QueryTestCaseBase { @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index cc21c73c81..13515057b0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -48,7 +48,6 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestSelectQuery extends QueryTestCaseBase { public TestSelectQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java index bd17daf735..c9c73391b9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSetSessionQuery.java @@ -21,13 +21,15 @@ import org.apache.tajo.IntegrationTest; import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; +import org.apache.tajo.catalog.CatalogUtil; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestSetSessionQuery extends QueryTestCaseBase { public TestSetSessionQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java index cce7168dc1..24b525d8fb 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java @@ -24,6 +24,7 @@ import org.apache.tajo.TajoTestingCluster; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes.Type; +import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.util.KeyValueSet; @@ -31,11 +32,11 @@ import org.junit.experimental.categories.Category; import java.sql.ResultSet; +import java.util.TimeZone; import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestSortQuery extends QueryTestCaseBase { public TestSortQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java index 888b8abe4e..ef57356c80 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java @@ -59,7 +59,6 @@ import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) -@net.jcip.annotations.NotThreadSafe public class TestTablePartitions extends QueryTestCaseBase { private NodeType nodeType; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java index dea5a4bb66..bc643aba03 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java @@ -24,7 +24,6 @@ import java.sql.ResultSet; -@net.jcip.annotations.NotThreadSafe public class TestTableSubQuery extends QueryTestCaseBase { public TestTableSubQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java index 6ccefb14ab..1be21e4486 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java @@ -31,7 +31,6 @@ import static org.junit.Assert.fail; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestTruncateTable extends QueryTestCaseBase { public TestTruncateTable() { super(TajoConstants.DEFAULT_DATABASE_NAME); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java index d481d4deb6..03c9c24b01 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java @@ -37,7 +37,6 @@ * - O - order by */ @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestUnionQuery extends QueryTestCaseBase { public TestUnionQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java index e6df40aa6b..9993992aa9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java @@ -34,7 +34,6 @@ import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestWindowQuery extends QueryTestCaseBase { public TestWindowQuery() { diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java index 81cdbb81d2..8ee6755e11 100644 --- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java +++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java @@ -33,7 +33,6 @@ import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestTajoDatabaseMetaData extends QueryTestCaseBase { private static InetSocketAddress tajoMasterAddress; diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java index 67263bb50e..ad74046712 100644 --- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java +++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java @@ -38,7 +38,6 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestTajoJdbc extends QueryTestCaseBase { private static InetSocketAddress tajoMasterAddress; diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java index ac1a85a9a4..e8d59d0d5e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java @@ -18,10 +18,49 @@ package org.apache.tajo.master; -import org.apache.tajo.QueryTestCaseBase; +import static org.junit.Assert.*; +import static org.hamcrest.CoreMatchers.*; + +import java.io.File; +import java.sql.ResultSet; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; + +import org.apache.tajo.*; +import org.apache.tajo.algebra.Expr; +import org.apache.tajo.benchmark.TPCH; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.client.ResultSetUtil; +import org.apache.tajo.client.TajoClient; +import org.apache.tajo.common.TajoDataTypes.Type; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.engine.parser.SQLAnalyzer; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.master.TajoMaster.MasterContext; +import org.apache.tajo.master.exec.NonForwardQueryResultScanner; +import org.apache.tajo.master.exec.NonForwardQueryResultSystemScanner; +import org.apache.tajo.plan.LogicalOptimizer; +import org.apache.tajo.plan.LogicalPlan; +import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.logical.LimitNode; +import org.apache.tajo.plan.logical.NodeType; +import org.apache.tajo.storage.RowStoreUtil; +import org.apache.tajo.storage.StorageConstants; +import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.util.KeyValueSet; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeDiagnosingMatcher; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; -@net.jcip.annotations.NotThreadSafe +import com.google.protobuf.ByteString; + public class TestNonForwardQueryResultSystemScanner extends QueryTestCaseBase { @Test public void testGetNextRowsForAggregateFunction() throws Exception { diff --git a/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java b/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java index 6a9cf59c50..dc3462065d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/rule/TestMasterRules.java @@ -18,6 +18,9 @@ package org.apache.tajo.master.rule; +import static org.junit.Assert.*; +import static org.hamcrest.CoreMatchers.*; + import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -25,20 +28,18 @@ import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.rule.*; +import org.apache.tajo.rule.EvaluationContext; +import org.apache.tajo.rule.EvaluationFailedException; +import org.apache.tajo.rule.EvaluationResult; import org.apache.tajo.rule.EvaluationResult.EvaluationResultCode; +import org.apache.tajo.rule.SelfDiagnosisRuleEngine; +import org.apache.tajo.rule.SelfDiagnosisRuleSession; import org.apache.tajo.rule.base.TajoConfValidationRule; import org.apache.tajo.util.CommonTestingUtil; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; - -@net.jcip.annotations.NotThreadSafe public class TestMasterRules { private static Path rootFilePath; diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java index 921fc1a11a..ab5375c6d1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java @@ -36,7 +36,6 @@ import static org.junit.Assert.*; @Category(IntegrationTest.class) -@net.jcip.annotations.NotThreadSafe public class TestTaskStatusUpdate extends QueryTestCaseBase { public TestTaskStatusUpdate() { diff --git a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java index cb2b9eecc5..f442bde55a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java +++ b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java @@ -42,7 +42,6 @@ import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestHistoryWriterReader extends QueryTestCaseBase { public static final String HISTORY_DIR = "/tmp/tajo-test-history"; TajoConf tajoConf; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index ea30e614a8..947773f72c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -42,8 +42,6 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; import static org.junit.Assert.*; - -@net.jcip.annotations.NotThreadSafe public class TestNodeResourceManager { private MockNodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 8e1e537f64..51a7771c8b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -35,7 +35,6 @@ import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestNodeStatusUpdater { private NodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index d696d8ecf5..ca19f6f5f9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -45,7 +45,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -@net.jcip.annotations.NotThreadSafe public class TestTaskExecutor { private NodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index c479af09fd..73474cfcee 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -44,7 +44,6 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestTaskManager { private NodeResourceManager resourceManager; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java index 2b7d37acb4..c572eee252 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestClusterResource.java @@ -18,6 +18,15 @@ package org.apache.tajo.ws.rs.resources; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.logging.Logger; + +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.core.GenericType; + import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf.ConfVars; @@ -28,16 +37,8 @@ import org.junit.Before; import org.junit.Test; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.core.GenericType; -import java.net.URI; -import java.util.List; -import java.util.Map; - import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestClusterResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java index 698ce090b4..59c53f1766 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestDatabasesResource.java @@ -18,6 +18,18 @@ package org.apache.tajo.ws.rs.resources; +import java.net.URI; +import java.util.Collection; +import java.util.Map; + +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.GenericType; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; + import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf.ConfVars; @@ -30,20 +42,8 @@ import org.junit.Before; import org.junit.Test; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.client.Entity; -import javax.ws.rs.core.GenericType; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; -import java.net.URI; -import java.util.Collection; -import java.util.Map; - import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestDatabasesResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java index f1346da901..7c7aeaed83 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestFunctionsResource.java @@ -38,7 +38,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -@net.jcip.annotations.NotThreadSafe public class TestFunctionsResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java index f2f8d8213d..5d0b150c72 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResource.java @@ -47,7 +47,6 @@ import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestQueryResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java index 51ba5d3951..fec162688c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestQueryResultResource.java @@ -54,7 +54,6 @@ import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestQueryResultResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java index 43e88b8a7e..7b293d840e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestSessionsResource.java @@ -18,6 +18,18 @@ package org.apache.tajo.ws.rs.resources; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; + +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.GenericType; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; + import org.apache.tajo.QueryTestCaseBase; import org.apache.tajo.TajoConstants; import org.apache.tajo.conf.TajoConf.ConfVars; @@ -30,20 +42,8 @@ import org.junit.Before; import org.junit.Test; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.client.Entity; -import javax.ws.rs.core.GenericType; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; -import java.net.URI; -import java.util.HashMap; -import java.util.Map; - import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestSessionsResource extends QueryTestCaseBase { private URI restServiceURI; diff --git a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java index 7f9b7642e8..3f2bd505fe 100644 --- a/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java +++ b/tajo-core/src/test/java/org/apache/tajo/ws/rs/resources/TestTablesResource.java @@ -44,7 +44,6 @@ import static org.junit.Assert.*; -@net.jcip.annotations.NotThreadSafe public class TestTablesResource extends QueryTestCaseBase { private URI restServiceURI; From 45840ea679bae9961a29a75f875f4b710f6a7804 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sun, 12 Jul 2015 22:31:37 +0900 Subject: [PATCH 54/80] TAJO-1397 --- .travis.yml | 1 - tajo-core/pom.xml | 2 +- .../tajo/querymaster/DefaultTaskScheduler.java | 13 +++++++++---- .../org/apache/tajo/querymaster/QueryMaster.java | 11 +++++++++++ .../java/org/apache/tajo/querymaster/Stage.java | 2 +- 5 files changed, 22 insertions(+), 7 deletions(-) diff --git a/.travis.yml b/.travis.yml index 7eb94a3b84..dd8485674e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -35,7 +35,6 @@ notifications: irc: "chat.freenode.net#tajo" before_install: ulimit -t 514029 -n 3000 -before_install: ulimit -a install: ./dev-support/travis-install-dependencies.sh script: diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 54a980475e..6ad6956730 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -585,7 +585,7 @@ ${maven.fork.count} true false - -Xms512m -Xmx1200m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 + -Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 true true diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index a448fbd5ce..4210d16708 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -135,7 +135,8 @@ public void stop() { schedulingThread.notifyAll(); } } - + candidateWorkers.clear(); + scheduledRequests.clear(); LOG.info("Task Scheduler stopped"); super.stop(); } @@ -659,6 +660,13 @@ private class ScheduledRequests { private Map leafTaskHostMapping = Maps.newConcurrentMap(); private final Map> leafTasksRackMapping = Maps.newConcurrentMap(); + protected void clear() { + leafTasks.clear(); + nonLeafTasks.clear(); + leafTaskHostMapping.clear(); + leafTasksRackMapping.clear(); + } + private void addLeafTask(TaskAttemptToSchedulerEvent event) { TaskAttempt taskAttempt = event.getTaskAttempt(); List locations = taskAttempt.getTask().getDataLocations(); @@ -707,8 +715,6 @@ public int nonLeafTaskNum() { return nonLeafTasks.size(); } - public Set assignedRequest = new HashSet(); - private TaskAttemptId allocateLocalTask(String host){ HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); @@ -918,7 +924,6 @@ public void assignToLeafTasks(LinkedList taskRequests) { } catch (Exception e) { LOG.error(e); } - assignedRequest.add(attemptId); scheduledObjectNum--; } else { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index bc9bbc585e..81a0a7ef33 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -93,6 +93,8 @@ public class QueryMaster extends CompositeService implements EventHandler { private ExecutorService eventExecutor; + private ExecutorService singleEventExecutor; + public QueryMaster(TajoWorker.WorkerContext workerContext) { super(QueryMaster.class.getName()); this.workerContext = workerContext; @@ -132,6 +134,7 @@ public void serviceStart() throws Exception { finishedQueryMasterTaskCleanThread.start(); eventExecutor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); + singleEventExecutor = Executors.newSingleThreadExecutor(); super.serviceStart(); LOG.info("QueryMaster started"); } @@ -156,6 +159,10 @@ public void serviceStop() throws Exception { eventExecutor.shutdown(); } + if(singleEventExecutor != null){ + singleEventExecutor.shutdown(); + } + super.serviceStop(); LOG.info("QueryMaster stopped"); } @@ -241,6 +248,10 @@ public ExecutorService getEventExecutor(){ return eventExecutor; } + public ExecutorService getSingleEventExecutor(){ + return singleEventExecutor; + } + public AsyncDispatcher getDispatcher() { return dispatcher; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index d54766ba00..f1d0d1622a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -810,7 +810,7 @@ public StageState transition(final Stage stage, StageEvent stageEvent) { setShuffleIfNecessary(stage, channel); initTaskScheduler(stage); // execute pre-processing asyncronously - stage.getContext().getQueryMasterContext().getEventExecutor() + stage.getContext().getQueryMasterContext().getSingleEventExecutor() .submit(new Runnable() { @Override public void run() { From a546fc080779ed962cca700cb804ef946b3658b6 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 13 Jul 2015 12:38:20 +0900 Subject: [PATCH 55/80] chang history replication to hdfs default --- .../src/main/java/org/apache/tajo/conf/TajoConf.java | 2 -- .../org/apache/tajo/util/history/HistoryWriter.java | 11 ++++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) 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 3b8df2983f..4997079bde 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 @@ -272,8 +272,6 @@ public static enum ConfVars implements ConfigKey { HISTORY_QUERY_DIR("tajo.history.query.dir", STAGING_ROOT_DIR.defaultVal + "/history"), HISTORY_TASK_DIR("tajo.history.task.dir", "file:///tmp/tajo-${user.name}/history"), HISTORY_EXPIRY_TIME_DAY("tajo.history.expiry-time-day", 7), - HISTORY_QUERY_REPLICATION("tajo.history.query.replication", 1, Validators.min("1")), - HISTORY_TASK_REPLICATION("tajo.history.task.replication", 1, Validators.min("1")), // Misc ------------------------------------------------------------------- // Fragment diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java index 25938586ef..c51b4e64a6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java @@ -53,6 +53,9 @@ */ public class HistoryWriter extends AbstractService { private static final Log LOG = LogFactory.getLog(HistoryWriter.class); + public static final String HISTORY_QUERY_REPLICATION = "tajo.history.query.replication"; + public static final String HISTORY_TASK_REPLICATION = "tajo.history.task.replication"; + public static final String QUERY_LIST = "query-list"; public static final String QUERY_DETAIL = "query-detail"; public static final String HISTORY_FILE_POSTFIX = ".hist"; @@ -87,13 +90,15 @@ public void serviceInit(Configuration conf) throws Exception { if (!(conf instanceof TajoConf)) { throw new IllegalArgumentException("conf should be a TajoConf type."); } - tajoConf = (TajoConf)conf; + tajoConf = (TajoConf) conf; historyParentPath = tajoConf.getQueryHistoryDir(tajoConf); taskHistoryParentPath = tajoConf.getTaskHistoryDir(tajoConf); writerThread = new WriterThread(); historyCleaner = new HistoryCleaner(tajoConf, isMaster); - queryReplication = (short) tajoConf.getIntVar(TajoConf.ConfVars.HISTORY_QUERY_REPLICATION); - taskReplication = (short) tajoConf.getIntVar(TajoConf.ConfVars.HISTORY_TASK_REPLICATION); + queryReplication = (short) tajoConf.getInt(HISTORY_QUERY_REPLICATION, + FileSystem.get(tajoConf).getDefaultReplication(historyParentPath)); + taskReplication = (short) tajoConf.getInt(HISTORY_TASK_REPLICATION, + FileSystem.get(tajoConf).getDefaultReplication(taskHistoryParentPath)); super.serviceInit(conf); } From a23d8077cae28baab72ed242f86fb3a3ba70f842 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 13 Jul 2015 14:27:19 +0900 Subject: [PATCH 56/80] fix task history can't display and upgrade netty to 4.0.29 --- tajo-core/src/main/resources/webapps/admin/querytasks.jsp | 6 ++++-- tajo-project/pom.xml | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp index 382004d4d6..d41e8a8813 100644 --- a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp @@ -91,24 +91,26 @@ long totalWriteBytes = 0; long totalWriteRows = 0; + List allTasks = null; if (stage != null) { totalInputBytes = stage.getTotalInputBytes(); totalReadBytes = stage.getTotalReadBytes(); totalReadRows = stage.getTotalReadRows(); totalWriteBytes = stage.getTotalWriteBytes(); totalWriteRows = stage.getTotalWriteRows(); + allTasks = stage.getTasks(); } - List allTasks = reader.getTaskHistory(queryId, ebId); - int numTasks = allTasks.size(); int numShuffles = 0; float totalProgress = 0.0f; + int numTasks = 0; if (allTasks != null) { for(TaskHistory eachTask: allTasks) { totalProgress += eachTask.getProgress(); numShuffles = eachTask.getNumShuffles(); } + numTasks = allTasks.size(); } int currentPage = 1; diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml index a98cc13b5c..d7e2f917c0 100644 --- a/tajo-project/pom.xml +++ b/tajo-project/pom.xml @@ -38,7 +38,7 @@ 0.11.0-SNAPSHOT 0.98.7-hadoop2 1.1.0 - 4.0.25.Final + 4.0.29.Final 2.6 ${project.parent.relativePath}/.. src/main/hadoop-${hadoop.version} From 9df50c7479b97cf257aab8cae4b4716319c597e8 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 13 Jul 2015 15:42:16 +0900 Subject: [PATCH 57/80] fix webUI NPE --- .../org/apache/tajo/util/history/HistoryReader.java | 6 +++++- .../src/main/resources/webapps/admin/querytasks.jsp | 6 ++---- tajo-core/src/main/resources/webapps/admin/task.jsp | 12 +++++++----- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java index f4719b27f2..a8737c97ac 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java @@ -192,7 +192,11 @@ public QueryHistory getQueryHistory(String queryId, long startTime) throws IOExc } public List getTaskHistory(String queryId, String ebId) throws IOException { - Path queryHistoryFile = getQueryHistoryFilePath(queryId, 0); + return getTaskHistory(queryId, ebId, 0); + } + + public List getTaskHistory(String queryId, String ebId, long startTime) throws IOException { + Path queryHistoryFile = getQueryHistoryFilePath(queryId, startTime); if (queryHistoryFile == null) { return new ArrayList(); } diff --git a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp index d41e8a8813..27b189b5e1 100644 --- a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp @@ -91,26 +91,24 @@ long totalWriteBytes = 0; long totalWriteRows = 0; - List allTasks = null; if (stage != null) { totalInputBytes = stage.getTotalInputBytes(); totalReadBytes = stage.getTotalReadBytes(); totalReadRows = stage.getTotalReadRows(); totalWriteBytes = stage.getTotalWriteBytes(); totalWriteRows = stage.getTotalWriteRows(); - allTasks = stage.getTasks(); } + List allTasks = reader.getTaskHistory(queryId, ebId, NumberUtils.toLong(startTime, 0)); + int numTasks = allTasks.size(); int numShuffles = 0; float totalProgress = 0.0f; - int numTasks = 0; if (allTasks != null) { for(TaskHistory eachTask: allTasks) { totalProgress += eachTask.getProgress(); numShuffles = eachTask.getNumShuffles(); } - numTasks = allTasks.size(); } int currentPage = 1; diff --git a/tajo-core/src/main/resources/webapps/admin/task.jsp b/tajo-core/src/main/resources/webapps/admin/task.jsp index 63e306fafe..d3014b15e6 100644 --- a/tajo-core/src/main/resources/webapps/admin/task.jsp +++ b/tajo-core/src/main/resources/webapps/admin/task.jsp @@ -19,12 +19,13 @@ %> <%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%> -<%@ page import="org.apache.tajo.util.JSPUtil" %> -<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> -<%@ page import="java.text.SimpleDateFormat" %> +<%@ page import="org.apache.commons.lang.math.NumberUtils" %> <%@ page import="org.apache.tajo.master.TajoMaster" %> +<%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.history.HistoryReader" %> <%@ page import="org.apache.tajo.util.history.TaskHistory" %> +<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> +<%@ page import="java.text.SimpleDateFormat" %> <%@ page import="java.util.List" %> <% @@ -33,15 +34,16 @@ String queryId = request.getParameter("queryId"); String ebId = request.getParameter("ebid"); - + String startTime = request.getParameter("startTime"); String status = request.getParameter("status"); + if(status == null || status.isEmpty() || "null".equals(status)) { status = "ALL"; } String taskAttemptId = request.getParameter("taskAttemptId"); - List allTasks = reader.getTaskHistory(queryId, ebId); + List allTasks = reader.getTaskHistory(queryId, ebId, NumberUtils.toLong(startTime, 0)); TaskHistory task = null; for(TaskHistory eachTask: allTasks) { From 352932b42386a6c28ab1709b619d5085d3c4e064 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 13 Jul 2015 17:06:57 +0900 Subject: [PATCH 58/80] add submitted query in tajo-web --- .../scheduler/AbstractQueryScheduler.java | 7 +++ .../master/scheduler/SimpleScheduler.java | 15 +++---- .../scheduler/TajoResourceScheduler.java | 6 +++ .../main/resources/webapps/admin/query.jsp | 44 ++++++++++++++++--- 4 files changed, 59 insertions(+), 13 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java index deb742f09e..29694c6d86 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java @@ -29,10 +29,12 @@ public abstract class AbstractQueryScheduler extends AbstractService implements protected final NodeResource clusterResource; protected final NodeResource minResource; protected final NodeResource maxResource; + protected final NodeResource qmMinResource; public AbstractQueryScheduler(String name) { super(name); this.minResource = NodeResources.createResource(0); + this.qmMinResource = NodeResources.createResource(0); this.maxResource = NodeResources.createResource(0); this.clusterResource = NodeResources.createResource(0); } @@ -52,6 +54,11 @@ public NodeResource getMaximumResourceCapability() { return maxResource; } + @Override + public NodeResource getQMMinimumResourceCapability() { + return qmMinResource; + } + public abstract int getRunningQuery(); public abstract ResourceCalculator getResourceCalculator(); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 10983eac87..b1c6e809fe 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -81,8 +81,8 @@ public SimpleScheduler(TajoMaster.MasterContext context) { } private void initScheduler(TajoConf conf) { - int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); - this.minResource.setMemory(minQMMem).setVirtualCores(1); + this.minResource.setMemory(conf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY)).setVirtualCores(1); + this.qmMinResource.setMemory(conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY)).setVirtualCores(1); updateResource(); this.queryProcessor.setName("Query Processor"); } @@ -135,9 +135,7 @@ public ResourceCalculator getResourceCalculator() { } private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceRequest(QueryInfo queryInfo) { - int qmMemory = tajoConf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); - NodeResource qmResource = NodeResources.createResource(qmMemory); - + NodeResource qmResource = getQMMinimumResourceCapability(); int containers = 1; Set assignedQMNodes = Sets.newHashSet(assignedQueryMasterMap.values()); @@ -337,10 +335,10 @@ public void run() { } //TODO get by assigned queue int maxAvailable = getResourceCalculator().computeAvailableContainers( - getMaximumResourceCapability(), getMinimumResourceCapability()); + getMaximumResourceCapability(), getQMMinimumResourceCapability()); // check maximum running queries - if (assignedQueryMasterMap.size() * 2 > maxAvailable) { + if (assignedQueryMasterMap.size() * 2 >= maxAvailable) { queryQueue.add(query); synchronized (this) { try { @@ -359,6 +357,8 @@ public void run() { if(allocation.size() == 0) { queryQueue.add(query); + LOG.info("No Available Resources for QueryMaster :" + queryInfo.getQueryId() + "," + queryInfo); + synchronized (this) { try { this.wait(100); @@ -366,7 +366,6 @@ public void run() { LOG.fatal(e); } } - LOG.info("No Available Resources for QueryMaster :" + queryInfo.getQueryId() + "," + queryInfo); } else { try { //if QM resource can't be allocated to a node, it should retry diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java index 170a6e8ca9..d73f28aed9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java @@ -44,6 +44,12 @@ public interface TajoResourceScheduler extends EventHandler { */ NodeResource getMinimumResourceCapability(); + /** + * Get minimum allocatable {@link NodeResource} of QueryMaster. + * @return minimum allocatable resource + */ + NodeResource getQMMinimumResourceCapability(); + /** * Get maximum allocatable {@link NodeResource}. * @return maximum allocatable resource diff --git a/tajo-core/src/main/resources/webapps/admin/query.jsp b/tajo-core/src/main/resources/webapps/admin/query.jsp index 495e0eef23..bc1a04b41a 100644 --- a/tajo-core/src/main/resources/webapps/admin/query.jsp +++ b/tajo-core/src/main/resources/webapps/admin/query.jsp @@ -38,11 +38,13 @@ InetSocketAddress socketAddress = new InetSocketAddress(masterName[0], Integer.parseInt(masterName[1])); String masterLabel = socketAddress.getAddress().getHostName()+ ":" + socketAddress.getPort(); - List runningQueries = + List submittedQueries = new ArrayList(master.getContext().getQueryJobManager().getSubmittedQueries()); + JSPUtil.sortQueryInProgress(submittedQueries, true); - runningQueries.addAll(master.getContext().getQueryJobManager().getRunningQueries()); - JSPUtil.sortQueryInProgress(runningQueries, true); + List runningQueries = + new ArrayList(master.getContext().getQueryJobManager().getRunningQueries()); + JSPUtil.sortQueryInProgress(runningQueries, true); int currentPage = 1; if (request.getParameter("page") != null && !request.getParameter("page").isEmpty()) { @@ -118,7 +120,39 @@ <%@ include file="header.jsp"%>

    Tajo Master: <%=masterLabel%> <%=JSPUtil.getMasterActiveLabel(master.getContext())%>

    -
    +

    +


    +

    Submitted Queries

    + <% + if(submittedQueries.isEmpty()) { + out.write("No submitted queries"); + } else { + %> +
    NoIdStatusProgressStartedRunning TimeHost
    NoIdStatusProgressStartedRunning TimeRetryHost
    <%=rowNo%><%=rowNo%> <%=eachTask.getId()%><%=eachTask.getLastAttemptStatus()%><%=JSPUtil.percentFormat(eachTask.getLastAttempt().getProgress())%>%<%=eachTask.getLaunchTime() == 0 ? "-" : df.format(eachTask.getLaunchTime())%><%=eachTask.getLastAttemptStatus()%><%=JSPUtil.percentFormat(eachTask.getLastAttempt().getProgress())%>%<%=eachTask.getLaunchTime() == 0 ? "-" : df.format(eachTask.getLaunchTime())%> <%=eachTask.getLaunchTime() == 0 ? "-" : eachTask.getRunningTime() + " ms"%><%=eachTask.getRetryCount()%> <%=taskHost%>
    System dir:<%=TajoConf.getSystemDir(master.getContext().getConf())%>
    Warehouse dir:<%=TajoConf.getWarehouseDir(master.getContext().getConf())%>
    Staging dir:<%=TajoConf.getDefaultRootStagingDir(master.getContext().getConf())%>
    Client Service:<%=NetUtils.normalizeInetSocketAddress(master.getTajoMasterClientService().getBindAddress())%>
    Catalog Service:<%=master.getCatalogServer().getCatalogServerName()%>
    Client Service:<%=master.getTajoMasterClientService().getBindAddress().getHostName() + ":" + master.getTajoMasterClientService().getBindAddress().getPort()%>
    Catalog Service:<%=master.getCatalogServer().getBindAddress().getHostName() + ":" + master.getCatalogServer().getBindAddress().getPort()%>
    Heap(Free/Total/Max): <%=Runtime.getRuntime().freeMemory()/1024/1024%> MB / <%=Runtime.getRuntime().totalMemory()/1024/1024%> MB / <%=Runtime.getRuntime().maxMemory()/1024/1024%> MB
    Configuration:detail...
    Environment:detail...
    + + <% + for(QueryInProgress eachQuery: submittedQueries) { + long time = System.currentTimeMillis() - eachQuery.getQueryInfo().getStartTime(); + %> + + + + + + + + + + + <% + } + %> +
    QueryIdQuery MasterSubmittedProgressTimeStatussqlKill Query
    <%=eachQuery.getQueryId()%><%=eachQuery.getQueryInfo().getQueryMasterHost()%><%=df.format(eachQuery.getQueryInfo().getStartTime())%><%=(int)(eachQuery.getQueryInfo().getProgress() * 100.0f)%>%<%=StringUtils.formatTime(time)%><%=eachQuery.getQueryInfo().getQueryState()%><%=eachQuery.getQueryInfo().getSql()%>
    + <% + } + %> +

    +


    Running Queries

    <% if(runningQueries.isEmpty()) { @@ -141,7 +175,7 @@ <%=StringUtils.formatTime(time)%> <%=eachQuery.getQueryInfo().getQueryState()%> <%=eachQuery.getQueryInfo().getSql()%> - + <% } From cf53e96d35124039cff5c473c879587581c50bbd Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 13 Jul 2015 17:59:44 +0900 Subject: [PATCH 59/80] cleanup unused codes --- .../querymaster/DefaultTaskScheduler.java | 173 +++++++----------- 1 file changed, 63 insertions(+), 110 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 4210d16708..b2581b5813 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -25,7 +25,6 @@ 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.EventHandler; import org.apache.hadoop.yarn.util.RackResolver; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.conf.TajoConf; @@ -52,7 +51,6 @@ import java.net.InetSocketAddress; import java.util.*; import java.util.Map.Entry; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -68,8 +66,8 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler { private volatile boolean isStopped; private ScheduledRequests scheduledRequests; - private TaskRequests taskRequests; + private int minTaskMemory; private int nextTaskId = 0; private int scheduledObjectNum = 0; boolean isLeaf; @@ -84,7 +82,7 @@ public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { @Override public void init(Configuration conf) { scheduledRequests = new ScheduledRequests(); - taskRequests = new TaskRequests(); + minTaskMemory = context.getMasterContext().getConf().getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); super.init(conf); } @@ -144,35 +142,38 @@ public void stop() { private Fragment[] fragmentsForNonLeafTask; private Fragment[] broadcastFragmentsForNonLeafTask; - LinkedList taskRequestEvents = new LinkedList(); public void schedule() { - reserveNodeResource(); - - if (taskRequests.size() > 0) { - if (scheduledRequests.leafTaskNum() > 0) { - LOG.debug("Try to schedule tasks with taskRequestEvents: " + - taskRequests.size() + ", LeafTask Schedule Request: " + - scheduledRequests.leafTaskNum()); - taskRequests.getTaskRequests(taskRequestEvents, - scheduledRequests.leafTaskNum()); - LOG.debug("Get " + taskRequestEvents.size() + " taskRequestEvents "); - if (taskRequestEvents.size() > 0) { - scheduledRequests.assignToLeafTasks(taskRequestEvents); - taskRequestEvents.clear(); + try { + LinkedList taskRequests = createTaskRequest(); + + if (remainingScheduledObjectNum() == 0) { + // all task is done, wait for stopping message + synchronized (schedulingThread) { + schedulingThread.wait(500); } - } - } + } else { + if (taskRequests.size() == 0) { + synchronized (schedulingThread) { + schedulingThread.wait(50); + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Get " + taskRequests.size() + " taskRequestEvents "); + } - if (taskRequests.size() > 0) { - if (scheduledRequests.nonLeafTaskNum() > 0) { - LOG.debug("Try to schedule tasks with taskRequestEvents: " + - taskRequests.size() + ", NonLeafTask Schedule Request: " + - scheduledRequests.nonLeafTaskNum()); - taskRequests.getTaskRequests(taskRequestEvents, - scheduledRequests.nonLeafTaskNum()); - scheduledRequests.assignToNonLeafTasks(taskRequestEvents); - taskRequestEvents.clear(); + if (isLeaf) { + if (scheduledRequests.leafTaskNum() > 0) { + scheduledRequests.assignToLeafTasks(taskRequests); + } + } else { + if (scheduledRequests.nonLeafTaskNum() > 0) { + scheduledRequests.assignToNonLeafTasks(taskRequests); + } + } + } } + } catch (Throwable e) { + LOG.error(e.getMessage(), e); } } @@ -253,96 +254,48 @@ private Set getWorkerIds(Collection hosts){ } - private void reserveNodeResource() { + protected LinkedList createTaskRequest() throws Exception { + LinkedList taskRequestEvents = new LinkedList(); - int taskMem = context.getMasterContext().getConf().getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); - NettyClientBase tmClient; - try { - ServiceTracker serviceTracker = - context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); - tmClient = RpcClientManager.getInstance(). - getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); - QueryCoordinatorProtocol.QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); - - CallFuture callBack = new CallFuture(); - QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = - QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); - request.setCapacity(NodeResources.createResource(taskMem, isLeaf ? 1 : 0).getProto()) - .setNumContainers(Math.max(remainingScheduledObjectNum(), 1)) - .setPriority(stage.getPriority()) - .setQueryId(context.getMasterContext().getQueryId().getProto()) - .setType(isLeaf ? QueryCoordinatorProtocol.ResourceType.LEAF : - QueryCoordinatorProtocol.ResourceType.INTERMEDIATE) - .setUserId(context.getMasterContext().getQueryContext().getUser()) - .setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()) - .addAllCandidateNodes(candidateWorkers) - .setQueue(context.getMasterContext().getQueryContext().get("queue", "default")); //TODO set queue - - masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); - QueryCoordinatorProtocol.NodeResourceResponseProto responseProto = callBack.get(); - - for (QueryCoordinatorProtocol.AllocationResourceProto proto : responseProto.getResourceList()) { - - TaskRequestEvent taskRequestEvent = new TaskRequestEvent(proto.getWorkerId(), proto, context.getBlockId()); - taskRequests.handle(taskRequestEvent); - } - - if(remainingScheduledObjectNum() == 0) { - // all task is assigned, wait for stopping message - synchronized (schedulingThread){ - schedulingThread.wait(500); - } - } else { - if(responseProto.getResourceCount() == 0) { - synchronized (schedulingThread){ - schedulingThread.wait(50); - } - } - } - } catch (Throwable e) { - LOG.error(e.getMessage(), e); + int requestContainerNum = Math.max(remainingScheduledObjectNum(), 1); + if (LOG.isDebugEnabled()) { + LOG.debug("Try to schedule task resources: " + requestContainerNum); } - } - - @Override - public int remainingScheduledObjectNum() { - return scheduledObjectNum; - } - private class TaskRequests implements EventHandler { - private final LinkedBlockingQueue taskRequestQueue = - new LinkedBlockingQueue(); + ServiceTracker serviceTracker = + context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); + NettyClientBase tmClient = RpcClientManager.getInstance(). + getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); + QueryCoordinatorProtocol.QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); - @Override - public void handle(TaskRequestEvent event) { - if(LOG.isDebugEnabled()){ - LOG.debug("TaskRequest: " + event.getResponseProto().getWorkerId() + "," + event.getExecutionBlockId()); - } + CallFuture callBack = new CallFuture(); + QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = + QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); + request.setCapacity(NodeResources.createResource(minTaskMemory, isLeaf ? 1 : 0).getProto()) + .setNumContainers(requestContainerNum) + .setPriority(stage.getPriority()) + .setQueryId(context.getMasterContext().getQueryId().getProto()) + .setType(isLeaf ? QueryCoordinatorProtocol.ResourceType.LEAF : + QueryCoordinatorProtocol.ResourceType.INTERMEDIATE) + .setUserId(context.getMasterContext().getQueryContext().getUser()) + .setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()) + .addAllCandidateNodes(candidateWorkers) + .setQueue(context.getMasterContext().getQueryContext().get("queue", "default")); //TODO set queue - if(isStopped) { - return; - } - int qSize = taskRequestQueue.size(); - if (qSize != 0 && qSize % 1000 == 0) { - LOG.info("Size of event-queue in DefaultTaskScheduler is " + qSize); - } - int remCapacity = taskRequestQueue.remainingCapacity(); - if (remCapacity < 1000) { - LOG.warn("Very low remaining capacity in the event-queue " - + "of DefaultTaskScheduler: " + remCapacity); - } + masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); + QueryCoordinatorProtocol.NodeResourceResponseProto + responseProto = callBack.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); - taskRequestQueue.add(event); + for (QueryCoordinatorProtocol.AllocationResourceProto proto : responseProto.getResourceList()) { + taskRequestEvents.add(new TaskRequestEvent(proto.getWorkerId(), proto, context.getBlockId())); } - public void getTaskRequests(final Collection taskRequests, - int num) { - taskRequestQueue.drainTo(taskRequests, num); - } + return taskRequestEvents; + } - public int size() { - return taskRequestQueue.size(); - } + @Override + public int remainingScheduledObjectNum() { + return scheduledObjectNum; } public void releaseTaskAttempt(TaskAttempt taskAttempt) { From 727f3fa47e34d613ad606fde203e5863d9367c3a Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 13 Jul 2015 23:13:02 +0900 Subject: [PATCH 60/80] remove hardcoded configuration --- .../java/org/apache/tajo/SessionVars.java | 2 +- .../java/org/apache/tajo/conf/TajoConf.java | 8 +++-- .../tajo/engine/query/TaskRequestImpl.java | 15 ++++----- .../tajo/master/rm/TajoResourceManager.java | 33 +++++++++++++++---- .../tajo/master/rm/TajoResourceTracker.java | 6 +++- .../master/rm/WorkerLivelinessMonitor.java | 2 +- .../master/scheduler/SimpleScheduler.java | 2 +- .../querymaster/DefaultTaskScheduler.java | 32 +++++++++++++----- .../apache/tajo/worker/NodeStatusUpdater.java | 2 +- .../tajo/worker/TestNodeStatusUpdater.java | 2 +- 10 files changed, 71 insertions(+), 33 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/SessionVars.java b/tajo-common/src/main/java/org/apache/tajo/SessionVars.java index 28fdb0b742..832a5b4030 100644 --- a/tajo-common/src/main/java/org/apache/tajo/SessionVars.java +++ b/tajo-common/src/main/java/org/apache/tajo/SessionVars.java @@ -108,7 +108,7 @@ public enum SessionVars implements ConfigKey { Boolean.class, Validators.bool()), QUERY_EXECUTE_PARALLEL(ConfVars.$QUERY_EXECUTE_PARALLEL_MAX, "Maximum parallel running of execution blocks for a query", - DEFAULT, Integer.class, Validators.min("0")), + DEFAULT, Integer.class, Validators.min("1")), // for physical Executors EXTSORT_BUFFER_SIZE(ConfVars.$EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE, "sort buffer size for external sort (mb)", DEFAULT, 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 4997079bde..b614d58115 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 @@ -152,7 +152,7 @@ public static enum ConfVars implements ConfigKey { // Resource tracker service RESOURCE_TRACKER_RPC_ADDRESS("tajo.resource-tracker.rpc.address", "localhost:26003", Validators.networkAddr()), - RESOURCE_TRACKER_HEARTBEAT_TIMEOUT("tajo.resource-tracker.heartbeat.timeout-secs", 120 * 1000), // seconds + RESOURCE_TRACKER_HEARTBEAT_TIMEOUT("tajo.resource-tracker.heartbeat.timeout-secs", 120), // seconds // QueryMaster resource TAJO_QUERYMASTER_MINIMUM_MEMORY("tajo.qm.resource.min.memory-mb", 500, Validators.min("64")), @@ -187,12 +187,15 @@ public static enum ConfVars implements ConfigKey { WORKER_HISTORY_EXPIRE_PERIOD("tajo.worker.history.expire-interval-minutes", 60), // 1 hours QUERYMASTER_CACHE_EXPIRE_PERIOD("tajo.qm.history.expire-interval-minutes", 10), // 10 mins - WORKER_HEARTBEAT_INTERVAL("tajo.worker.heartbeat.interval", 10 * 1000), // 10 sec + WORKER_HEARTBEAT_IDLE_INTERVAL("tajo.worker.heartbeat.idle.interval", 10 * 1000), // 10 sec + WORKER_HEARTBEAT_ACTIVE_INTERVAL("tajo.worker.heartbeat.active.interval", 1000), // 1 sec //Default query scheduler RESOURCE_SCHEDULER_CLASS("tajo.resource.scheduler", "org.apache.tajo.master.scheduler.SimpleScheduler", Validators.groups(Validators.notNull(), Validators.clazz())), + QUERYMASTER_TASK_SCHEDULER_DELAY("tajo.qm.task.task-scheduler.delay", 50), // 50 ms + // Catalog CATALOG_ADDRESS("tajo.catalog.client-rpc.address", "localhost:26005", Validators.networkAddr()), @@ -314,7 +317,6 @@ public static enum ConfVars implements ConfigKey { $GROUPBY_MULTI_LEVEL_ENABLED("tajo.dist-query.groupby.multi-level-aggr", true), - // WARN "tajo.yarn-rm.parallel-task-runner-launcher-num" should be set enough to avoid deadlock $QUERY_EXECUTE_PARALLEL_MAX("tajo.query.execute.parallel.max", 10), // for physical Executors diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java index b9e154bd71..afb6cc003d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java @@ -34,19 +34,18 @@ import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; public class TaskRequestImpl implements TaskRequest { - - private TaskAttemptId id; - private List fragments; - private String outputTable; + + private TaskAttemptId id; + private List fragments; + private String outputTable; private boolean isUpdated; private boolean clusteredOutput; private PlanProto.LogicalNodeTree plan; // logical node private Boolean interQuery; private List fetches; - private Boolean shouldDie; - private QueryContext queryContext; - private DataChannel dataChannel; - private Enforcer enforcer; + private QueryContext queryContext; + private DataChannel dataChannel; + private Enforcer enforcer; private String queryMasterHostAndPort; private TaskRequestProto proto = TajoWorkerProtocol.TaskRequestProto.getDefaultInstance(); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java index 4fd6880fb6..af3d711ed7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java @@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -31,11 +32,11 @@ import org.apache.tajo.master.TajoMaster; import org.apache.tajo.master.scheduler.AbstractQueryScheduler; import org.apache.tajo.master.scheduler.QuerySchedulingInfo; -import org.apache.tajo.master.scheduler.SimpleScheduler; import org.apache.tajo.master.scheduler.event.SchedulerEventType; import org.apache.tajo.util.TUtil; import java.io.IOException; +import java.lang.reflect.Constructor; import java.util.Collection; import java.util.Collections; import java.util.Map; @@ -47,6 +48,8 @@ public class TajoResourceManager extends CompositeService { /** class logger */ private static final Log LOG = LogFactory.getLog(TajoResourceManager.class); + protected static final Map> SCHEDULER_CLASS_CACHE = Maps.newHashMap(); + private TajoMaster.MasterContext masterContext; private TajoRMContext rmContext; @@ -85,7 +88,6 @@ public void serviceInit(Configuration conf) throws Exception { this.queryIdSeed = String.valueOf(System.currentTimeMillis()); - this.workerLivelinessMonitor = new WorkerLivelinessMonitor(this.rmContext.getDispatcher()); addIfService(this.workerLivelinessMonitor); @@ -95,14 +97,31 @@ public void serviceInit(Configuration conf) throws Exception { resourceTracker = new TajoResourceTracker(this, workerLivelinessMonitor); addIfService(resourceTracker); - //TODO configuable - scheduler = new SimpleScheduler(masterContext); + String schedulerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_SCHEDULER_CLASS); + scheduler = loadScheduler(schedulerClassName); + LOG.info("Loaded resource scheduler : " + scheduler.getClass()); addIfService(scheduler); rmContext.getDispatcher().register(SchedulerEventType.class, scheduler); super.serviceInit(systemConf); } + protected synchronized AbstractQueryScheduler loadScheduler(String schedulerClassName) throws Exception { + Class schedulerClass; + if (SCHEDULER_CLASS_CACHE.containsKey(schedulerClassName)) { + schedulerClass = SCHEDULER_CLASS_CACHE.get(schedulerClassName); + } else { + schedulerClass = (Class) Class.forName(schedulerClassName); + SCHEDULER_CLASS_CACHE.put(schedulerClassName, schedulerClass); + } + + Constructor + constructor = schedulerClass.getDeclaredConstructor(new Class[]{TajoMaster.MasterContext.class}); + constructor.setAccessible(true); + + return constructor.newInstance(new Object[]{masterContext}); + } + @InterfaceAudience.Private public static final class WorkerEventDispatcher implements EventHandler { @@ -126,15 +145,15 @@ public void handle(WorkerEvent event) { } } - @Deprecated + public Map getWorkers() { return ImmutableMap.copyOf(rmContext.getWorkers()); } - @Deprecated + public Map getInactiveWorkers() { return ImmutableMap.copyOf(rmContext.getInactiveWorkers()); } - @Deprecated + public Collection getQueryMasters() { return Collections.unmodifiableSet(rmContext.getQueryMasterWorker()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index b03b0e6951..a9e5abc2bf 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -68,6 +68,9 @@ public class TajoResourceTracker extends AbstractService implements TajoResource /** The bind address of RPC server of worker resource tracker */ private InetSocketAddress bindAddress; + /** worker heartbeat interval in query running */ + private int activeInterval; + public TajoResourceTracker(TajoResourceManager manager, WorkerLivelinessMonitor workerLivelinessMonitor) { super(TajoResourceTracker.class.getSimpleName()); this.manager = manager; @@ -79,6 +82,7 @@ public TajoResourceTracker(TajoResourceManager manager, WorkerLivelinessMonitor public void serviceInit(Configuration conf) throws Exception { TajoConf systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); + activeInterval = systemConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_ACTIVE_INTERVAL); String confMasterServiceAddr = systemConf.getVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS); InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr); @@ -188,7 +192,7 @@ public void nodeHeartbeat( } } finally { if(manager.getScheduler().getRunningQuery() > 0) { - response.setHeartBeatInterval(1000); //1 sec + response.setHeartBeatInterval(activeInterval); } done.run(response.setCommand(responseCommand).build()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java index 2751886e98..7638ec8255 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java @@ -43,7 +43,7 @@ public void serviceInit(Configuration conf) throws Exception { Preconditions.checkArgument(conf instanceof TajoConf); TajoConf systemConf = (TajoConf) conf; // milliseconds - int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.RESOURCE_TRACKER_HEARTBEAT_TIMEOUT); + int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.RESOURCE_TRACKER_HEARTBEAT_TIMEOUT) * 1000; setExpireInterval(expireIntvl); setMonitorInterval(expireIntvl/3); super.serviceInit(conf); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index b1c6e809fe..8d2adbadbc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -337,7 +337,7 @@ public void run() { int maxAvailable = getResourceCalculator().computeAvailableContainers( getMaximumResourceCapability(), getQMMinimumResourceCapability()); - // check maximum running queries + // check maximum parallel running queries if (assignedQueryMasterMap.size() * 2 >= maxAvailable) { queryQueue.add(query); synchronized (this) { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index b2581b5813..40a5b3d7e6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -46,12 +46,14 @@ import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.FetchImpl; import java.net.InetSocketAddress; import java.util.*; import java.util.Map.Entry; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; @@ -70,7 +72,10 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler { private int minTaskMemory; private int nextTaskId = 0; private int scheduledObjectNum = 0; - boolean isLeaf; + private boolean isLeaf; + private int schedulerDelay; + + //candidate workers for locality of high priority private Set candidateWorkers = Sets.newHashSet(); public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { @@ -81,8 +86,10 @@ public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { @Override public void init(Configuration conf) { + TajoConf tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); scheduledRequests = new ScheduledRequests(); - minTaskMemory = context.getMasterContext().getConf().getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); + minTaskMemory = tajoConf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); + schedulerDelay= tajoConf.getIntVar(TajoConf.ConfVars.QUERYMASTER_TASK_SCHEDULER_DELAY); super.init(conf); } @@ -142,7 +149,7 @@ public void stop() { private Fragment[] fragmentsForNonLeafTask; private Fragment[] broadcastFragmentsForNonLeafTask; - public void schedule() { + public void schedule() throws Exception{ try { LinkedList taskRequests = createTaskRequest(); @@ -154,7 +161,7 @@ public void schedule() { } else { if (taskRequests.size() == 0) { synchronized (schedulingThread) { - schedulingThread.wait(50); + schedulingThread.wait(schedulerDelay); } } else { if (LOG.isDebugEnabled()) { @@ -172,8 +179,8 @@ public void schedule() { } } } - } catch (Throwable e) { - LOG.error(e.getMessage(), e); + } catch (TimeoutException e) { + LOG.error(e.getMessage()); } } @@ -871,7 +878,10 @@ public void assignToLeafTasks(LinkedList taskRequests) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); cancellation++; } - LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); + + if(LOG.isDebugEnabled()) { + LOG.debug("Canceled requests: " + responseProto.getCancellationTaskCount() + " from " + addr); + } continue; } } catch (Exception e) { @@ -943,7 +953,8 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { } } - WorkerConnectionInfo connectionInfo = context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); + WorkerConnectionInfo connectionInfo = + context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); //TODO send batch request TajoWorkerProtocol.BatchAllocationRequestProto.Builder @@ -975,7 +986,10 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); cancellation++; } - LOG.info("Canceled requests: " + responseProto.getCancellationTaskCount()); + + if(LOG.isDebugEnabled()) { + LOG.debug("Canceled requests: " + responseProto.getCancellationTaskCount() + " from " + addr); + } continue; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index cb8a49a6d6..9e386d6754 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -79,7 +79,7 @@ public void serviceInit(Configuration conf) throws Exception { this.heartBeatRequestQueue = Queues.newLinkedBlockingQueue(); this.serviceTracker = ServiceTrackerFactory.get(tajoConf); this.workerContext.getNodeResourceManager().getDispatcher().register(NodeStatusEvent.EventType.class, this); - this.heartBeatInterval = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL); + this.heartBeatInterval = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_IDLE_INTERVAL); this.updaterThread = new StatusUpdaterThread(); this.updaterThread.setName("NodeStatusUpdater"); super.serviceInit(conf); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 51a7771c8b..4729c053f3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -83,7 +83,7 @@ public WorkerConnectionInfo getConnectionInfo() { } }; - conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL, 1000); + conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_IDLE_INTERVAL, 1000); dispatcher = new AsyncDispatcher(); resourceManager = new NodeResourceManager(dispatcher, workerContext); taskDispatcher = new AsyncDispatcher(); From e3a007863b03fd11bd9ac2cdc8f2a6389d29df7c Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 14 Jul 2015 11:56:53 +0900 Subject: [PATCH 61/80] add SimpleScheduler tests --- .../java/org/apache/tajo/conf/TajoConf.java | 2 +- .../master/scheduler/SimpleScheduler.java | 10 +- .../tajo/worker/NodeResourceManager.java | 4 +- .../master/scheduler/TestSimpleScheduler.java | 264 ++++++++++++++++-- 4 files changed, 252 insertions(+), 28 deletions(-) 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 b614d58115..c42046ab63 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 @@ -155,7 +155,7 @@ public static enum ConfVars implements ConfigKey { RESOURCE_TRACKER_HEARTBEAT_TIMEOUT("tajo.resource-tracker.heartbeat.timeout-secs", 120), // seconds // QueryMaster resource - TAJO_QUERYMASTER_MINIMUM_MEMORY("tajo.qm.resource.min.memory-mb", 500, Validators.min("64")), + QUERYMASTER_MINIMUM_MEMORY("tajo.qm.resource.min.memory-mb", 500, Validators.min("64")), // Worker task resource TASK_RESOURCE_MINIMUM_MEMORY("tajo.task.resource.min.memory-mb", 500, Validators.min("64")), diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 8d2adbadbc..c3b3831f8f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -50,7 +50,7 @@ public class SimpleScheduler extends AbstractQueryScheduler { private static final Log LOG = LogFactory.getLog(SimpleScheduler.class); - + private static final float PARALLEL_QUERY_LIMIT = 0.5f; private static final Comparator COMPARATOR = new SchedulingAlgorithms.FifoComparator(); private volatile boolean isStopped = false; @@ -82,7 +82,7 @@ public SimpleScheduler(TajoMaster.MasterContext context) { private void initScheduler(TajoConf conf) { this.minResource.setMemory(conf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY)).setVirtualCores(1); - this.qmMinResource.setMemory(conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY)).setVirtualCores(1); + this.qmMinResource.setMemory(conf.getIntVar(TajoConf.ConfVars.QUERYMASTER_MINIMUM_MEMORY)).setVirtualCores(1); updateResource(); this.queryProcessor.setName("Query Processor"); } @@ -330,15 +330,15 @@ public void run() { try { query = queryQueue.take(); } catch (InterruptedException e) { - e.printStackTrace(); + LOG.warn(e.getMessage(), e); break; } //TODO get by assigned queue int maxAvailable = getResourceCalculator().computeAvailableContainers( getMaximumResourceCapability(), getQMMinimumResourceCapability()); - // check maximum parallel running queries - if (assignedQueryMasterMap.size() * 2 >= maxAvailable) { + // check maximum parallel running queries. allow 50% parallel running + if (assignedQueryMasterMap.size() >= Math.floor(maxAvailable * PARALLEL_QUERY_LIMIT)) { queryQueue.add(query); synchronized (this) { try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index ba396ca0b3..fcb9978b88 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -181,7 +181,7 @@ private NodeResource createWorkerResource(TajoConf conf) { private void validateConf(TajoConf conf) { // validate node memory allocation setting int minMem = conf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); - int minQMMem = conf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY); + int minQMMem = conf.getIntVar(TajoConf.ConfVars.QUERYMASTER_MINIMUM_MEMORY); int maxMem = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); if (minMem <= 0 || minQMMem <= 0 || minMem + minQMMem > maxMem) { @@ -189,7 +189,7 @@ private void validateConf(TajoConf conf) { + " allocation configuration" + ", " + TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.varname + "=" + minMem - + ", " + TajoConf.ConfVars.TAJO_QUERYMASTER_MINIMUM_MEMORY.varname + + ", " + TajoConf.ConfVars.QUERYMASTER_MINIMUM_MEMORY.varname + "=" + minQMMem + ", " + TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname + "=" + maxMem + ", min and max should be greater than 0" diff --git a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java index c21a19939c..6dcbd95c28 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java @@ -18,57 +18,281 @@ package org.apache.tajo.master.scheduler; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import net.jcip.annotations.NotThreadSafe; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.tajo.QueryId; +import org.apache.tajo.QueryIdFactory; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.master.QueryInfo; -import org.apache.tajo.master.rm.TajoRMContext; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.master.rm.*; +import org.apache.tajo.master.scheduler.event.ResourceReserveSchedulerEvent; +import org.apache.tajo.master.scheduler.event.SchedulerEvent; +import org.apache.tajo.master.scheduler.event.SchedulerEventType; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.CallFuture; +import org.junit.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; + +import static org.apache.tajo.ipc.QueryCoordinatorProtocol.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@NotThreadSafe public class TestSimpleScheduler { - SimpleScheduler scheduler; - TajoRMContext rmContext; - AsyncDispatcher dispatcher; - TajoConf conf; + private CompositeService service; + private SimpleScheduler scheduler; + private TajoRMContext rmContext; + private AsyncDispatcher dispatcher; + private TajoConf conf; + private int workerNum = 3; + private NodeResource nodeResource; + private NodeResource totalResource; + private Semaphore barrier; + private int testDelay = 50; + private static ScheduledExecutorService executorService; + + @BeforeClass + public static void setupClass() { + executorService = Executors.newScheduledThreadPool(10); + } + + @AfterClass + public static void tearDownClass() { + executorService.shutdown(); + } @Before public void setup() { conf = new TajoConf(); - dispatcher = new AsyncDispatcher(); - dispatcher.init(conf); - dispatcher.start(); + nodeResource = NodeResource.createResource(1500, 2, 3); + service = new CompositeService(TestSimpleScheduler.class.getSimpleName()) { + + @Override + protected void serviceInit(Configuration conf) throws Exception { + dispatcher = new AsyncDispatcher(); + addService(dispatcher); - rmContext = new TajoRMContext(dispatcher); - scheduler = new MySimpleScheduler(rmContext); + rmContext = new TajoRMContext(dispatcher); + rmContext.getDispatcher().register(WorkerEventType.class, + new TajoResourceManager.WorkerEventDispatcher(rmContext)); + + barrier = new Semaphore(0); + scheduler = new MySimpleScheduler(rmContext, barrier); + addService(scheduler); + rmContext.getDispatcher().register(SchedulerEventType.class, scheduler); + + for (int i = 0; i < workerNum; i++) { + WorkerConnectionInfo conn = new WorkerConnectionInfo("host" + i, 28091 + i, 28092, 21000, 28093, 28080); + rmContext.getWorkers().putIfAbsent(conn.getId(), + new Worker(rmContext, NodeResources.clone(nodeResource), conn)); + rmContext.getDispatcher().getEventHandler().handle(new WorkerEvent(conn.getId(), WorkerEventType.STARTED)); + } + super.serviceInit(conf); + } + }; + service.init(conf); + service.start(); + + assertEquals(workerNum, rmContext.getWorkers().size()); + totalResource = NodeResources.createResource(0); + for(Worker worker : rmContext.getWorkers().values()) { + NodeResources.addTo(totalResource, worker.getTotalResourceCapability()); + } } @After public void tearDown() { - scheduler.stop(); - dispatcher.stop(); + service.stop(); } @Test - public void testReserveResource() { + public void testInitialCapacity() throws InterruptedException { + assertEquals(workerNum, scheduler.getNumClusterNodes()); + assertEquals(0, scheduler.getRunningQuery()); + + assertEquals(totalResource, scheduler.getMaximumResourceCapability()); + assertEquals(totalResource, scheduler.getClusterResource()); + + assertEquals(TajoConf.ConfVars.QUERYMASTER_MINIMUM_MEMORY.defaultIntVal, + scheduler.getQMMinimumResourceCapability().getMemory()); + + assertEquals(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY.defaultIntVal, + scheduler.getMinimumResourceCapability().getMemory()); + } + + @Test(timeout = 10000) + public void testSubmitOneQuery() throws InterruptedException { + QuerySchedulingInfo schedulingInfo = new QuerySchedulingInfo("default", + "user", + QueryIdFactory.newQueryId(System.nanoTime(), 0), + 1, + System.currentTimeMillis()); + + assertEquals(0, scheduler.getRunningQuery()); + + scheduler.submitQuery(schedulingInfo); + barrier.acquire(); + assertEquals(1, scheduler.getRunningQuery()); + + assertEquals(totalResource, scheduler.getMaximumResourceCapability()); + assertEquals(totalResource, + NodeResources.add(scheduler.getQMMinimumResourceCapability(), scheduler.getClusterResource())); + } + + @Test(timeout = 10000) + public void testMaximumSubmitQuery() throws InterruptedException { + assertEquals(0, scheduler.getRunningQuery()); + int maximumParallelQuery = scheduler.getResourceCalculator().computeAvailableContainers( + scheduler.getMaximumResourceCapability(), scheduler.getQMMinimumResourceCapability()); + + int testParallelNum = 10; + for (int i = 0; i < testParallelNum; i++) { + QuerySchedulingInfo schedulingInfo = new QuerySchedulingInfo("default", + "user", + QueryIdFactory.newQueryId(System.nanoTime(), 0), + 1, + System.currentTimeMillis()); + scheduler.submitQuery(schedulingInfo); + } + + barrier.acquire(); + // allow 50% parallel running + assertEquals(Math.floor(maximumParallelQuery * 0.5f), (double) scheduler.getRunningQuery(), 1.0f); + assertEquals(testParallelNum, scheduler.getRunningQuery() + scheduler.getQueryQueue().size()); + } + + @Test(timeout = 10000) + public void testReserveResource() throws InterruptedException, ExecutionException { + int requestNum = 3; + assertEquals(totalResource, scheduler.getMaximumResourceCapability()); + assertEquals(totalResource, scheduler.getClusterResource()); + + QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); + CallFuture callBack = new CallFuture(); + rmContext.getDispatcher().getEventHandler().handle(new ResourceReserveSchedulerEvent( + createResourceRequest(queryId, requestNum, new ArrayList()), callBack)); + + NodeResourceResponseProto responseProto = callBack.get(); + assertEquals(queryId, new QueryId(responseProto.getQueryId())); + assertEquals(requestNum, responseProto.getResourceCount()); + + NodeResource allocations = NodeResources.createResource(0); + for (AllocationResourceProto resourceProto : responseProto.getResourceList()) { + NodeResources.addTo(allocations, new NodeResource(resourceProto.getResource())); + } + + assertEquals(NodeResources.subtract(totalResource, allocations), scheduler.getClusterResource()); + } + + @Test(timeout = 10000) + public void testReserveResourceWithWorkerPriority() throws InterruptedException, ExecutionException { + int requestNum = 2; + assertEquals(totalResource, scheduler.getMaximumResourceCapability()); + assertEquals(totalResource, scheduler.getClusterResource()); + + List targetWorkers = Lists.newArrayList(); + Map.Entry workerEntry = rmContext.getWorkers().entrySet().iterator().next(); + targetWorkers.add(workerEntry.getKey()); + + NodeResource expectResource = NodeResources.multiply(scheduler.getMinimumResourceCapability(), requestNum); + assertTrue(NodeResources.fitsIn(expectResource, workerEntry.getValue().getAvailableResource())); + + QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); + NodeResourceRequestProto requestProto = createResourceRequest(queryId, requestNum, targetWorkers); + CallFuture callBack = new CallFuture(); + rmContext.getDispatcher().getEventHandler().handle(new ResourceReserveSchedulerEvent( + requestProto, callBack)); + + NodeResourceResponseProto responseProto = callBack.get(); + assertEquals(queryId, new QueryId(responseProto.getQueryId())); + assertEquals(requestNum, responseProto.getResourceCount()); + + for (AllocationResourceProto resourceProto : responseProto.getResourceList()) { + assertEquals(workerEntry.getKey().intValue(), resourceProto.getWorkerId()); + } + } + + private NodeResourceRequestProto + createResourceRequest(QueryId queryId, int containerNum, List candidateWorkers) { + NodeResourceRequestProto.Builder request = + NodeResourceRequestProto.newBuilder(); + request.setCapacity(scheduler.getMinimumResourceCapability().getProto()) + .setNumContainers(containerNum) + .setPriority(1) + .setQueryId(queryId.getProto()) + .setType(ResourceType.LEAF) + .setUserId("test user") + .setRunningTasks(0) + .addAllCandidateNodes(candidateWorkers) + .setQueue("default"); + return request.build(); } class MySimpleScheduler extends SimpleScheduler { + Semaphore barrier; + Map queryInfoMap = Maps.newHashMap(); - public MySimpleScheduler(TajoRMContext rmContext) { + public MySimpleScheduler(TajoRMContext rmContext, Semaphore barrier) { super(null, rmContext); + this.barrier = barrier; + } + + @Override + public void submitQuery(QuerySchedulingInfo schedulingInfo) { + queryInfoMap.put(schedulingInfo.getQueryId(), new QueryInfo(schedulingInfo.getQueryId()) { + QueryContext context; + @Override + public QueryContext getQueryContext() { + if(context == null) { + context = new QueryContext(conf); + context.setUser("user"); + } + return context; + } + }); + super.submitQuery(schedulingInfo); } @Override - protected boolean startQuery(QueryId queryId, QueryCoordinatorProtocol.AllocationResourceProto allocation) { + protected boolean startQuery(QueryId queryId, final AllocationResourceProto allocation) { + executorService.schedule(new Runnable() { + @Override + public void run() { + barrier.release(); + NodeResources.addTo(rmContext.getWorkers().get(allocation.getWorkerId()).getAvailableResource(), + new NodeResource(allocation.getResource())); + rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); + } + }, testDelay, TimeUnit.MILLISECONDS); return true; } + @Override + public void handle(SchedulerEvent event) { + super.handle(event); + barrier.release(); + } + @Override protected QueryInfo getQueryInfo(QueryId queryId) { - return new QueryInfo(queryId); + return queryInfoMap.get(queryId); + } + + @Override + public void stopQuery(QueryId queryId) { + queryInfoMap.remove(queryId); + super.stopQuery(queryId); } } } From 5ee2de0c8d00128a445a327bfc5637e9f1f55e44 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 14 Jul 2015 15:38:12 +0900 Subject: [PATCH 62/80] update documents --- .../configuration/worker_configuration.rst | 94 +++++++++++++------ 1 file changed, 67 insertions(+), 27 deletions(-) diff --git a/tajo-docs/src/main/sphinx/configuration/worker_configuration.rst b/tajo-docs/src/main/sphinx/configuration/worker_configuration.rst index 2713948f44..0756c0a022 100644 --- a/tajo-docs/src/main/sphinx/configuration/worker_configuration.rst +++ b/tajo-docs/src/main/sphinx/configuration/worker_configuration.rst @@ -43,21 +43,26 @@ Worker Resources ========================================================== Each worker can execute multiple tasks simultaneously. -In Tajo, users can specify the total size of memory and the number of disks for each worker. Available resources affect how many tasks are executed simultaneously. + +In Tajo, users can specify the number of cpu cores, the total size of memory and the number of disks for each worker. Available resources affect how many tasks are executed simultaneously. +CPU cores are a unit for expressing CPU parallelism, the unit for memory is megabytes and the unit for disks is the number of disk In order to specify the resource capacity of each worker, you should add the following configs to ``tajo-site.xml`` : -================================= ========================== =================== ========================= - property name description value type default value -================================= ========================== =================== ========================= - tajo.worker.resource.cpu-cores the number of cpu cores integer 1 - tajo.worker.resource.memory-mb memory size (MB) integer 1024 - tajo.worker.resource.disks the number of disks integer 1 -================================= ========================== =================== ========================= +=================================== ============= ====================== ================================= + property name value type default value description +=================================== ============= ====================== ================================= + tajo.worker.resource.cpu-cores Integer available cpu-cores the number of cpu cores + tajo.worker.resource.memory-mb Integer available jvm heap memory size (MB) + tajo.worker.resource.disks Integer 1 the number of disks + tajo.task.resource.min.memory-mb Integer 500 minimum allocatable memory per task + tajo.qm.resource.min.memory-mb Integer 500 minimum allocatable memory per query +=================================== ============= ====================== ================================= .. note:: - Currently, QueryMaster requests 512MB memory and 0.5 disk per task for the backward compatibility. + Currently, QueryMaster requests 500MB memory and 1 cpu-core per task for the backward compatibility. + If you want to give more memory, you can set to ``tajo.qm.resource.min.memory-mb`` .. note:: @@ -68,7 +73,14 @@ In order to specify the resource capacity of each worker, you should add the fol Example ------------ -Assume that you want to give 5120 MB memory, 4 disks, and 24 cores on each worker. The example configuration is as follows: +Assume that you want to give 15GB Jvm heap, 1GB memory per task, 4 disks, and 12 cores on each worker. The example configuration is as follows: + +``tajo-env.sh`` + +.. code-block:: bash + + export TAJO_WORKER_HEAPSIZE=15000 + ``tajo-site.xml`` @@ -76,35 +88,63 @@ Assume that you want to give 5120 MB memory, 4 disks, and 24 cores on each worke tajo.worker.resource.tajo.worker.resource.cpu-cores - 24 + 12 - tajo.worker.resource.memory-mb - 5120 + tajo.task.resource.min.memory-mb + 1000 - tajo.worker.resource.tajo.worker.resource.disks - 4.0 - + tajo.worker.resource.disks + 4 + + --------------------- - Dedicated Mode --------------------- -Tajo provides a dedicated mode that allows each worker in a Tajo cluster to use whole available system resources including cpu-cores, memory, and disks. For this mode, a user should add the following config to ``tajo-site.xml`` : +* Example with HDFS + +``tajo-env.sh`` + +.. code-block:: bash + + export TAJO_WORKER_HEAPSIZE=15000 + + +``tajo-site.xml`` .. code-block:: xml + + tajo.task.resource.min.memory-mb + 1000 + + - tajo.worker.resource.dedicated + tajo.worker.resource.dfs-dir-aware true -In addition, it can limit the memory capacity used for Tajo worker as follows: -=============================================== ================================================ =================== ======================= - property name description value type default value -=============================================== ================================================ =================== ======================= - tajo.worker.resource.dedicated-memory-ratio how much memory to be used in whole memory float 0.8 -=============================================== ================================================ =================== ======================= \ No newline at end of file +* Example with S3 + +``tajo-env.sh`` + +.. code-block:: bash + + export TAJO_WORKER_HEAPSIZE=15000 + + +``tajo-site.xml`` + +.. code-block:: xml + + + tajo.task.resource.min.memory-mb + 1000 + + + + tajo.worker.resource.disk.parallel-execution.num + 4 + \ No newline at end of file From bb07308d1bb4701d0b0ccee7758842b61270564a Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 14 Jul 2015 17:30:49 +0900 Subject: [PATCH 63/80] remove findbug warnings --- .../NonForwardQueryResultSystemScanner.java | 3 +-- .../master/scheduler/QuerySchedulingInfo.java | 20 +++++++++++++++++++ .../tajo/worker/NodeResourceManager.java | 4 ++-- .../apache/tajo/worker/NodeStatusUpdater.java | 11 +++------- .../org/apache/tajo/worker/TaskManager.java | 7 +++++-- 5 files changed, 31 insertions(+), 14 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java index 7d90498386..173c270aed 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java @@ -424,8 +424,7 @@ private List getAllPartitions(Schema outSchema) { private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { List columns = outSchema.getRootColumns(); Tuple aTuple = new VTuple(outSchema.size()); - NodeResource aResource = aWorker.getAvailableResource(); - + for (int fieldId = 0; fieldId < columns.size(); fieldId++) { Column column = columns.get(fieldId); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java index bb38ad44af..16d424c9df 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java @@ -18,6 +18,7 @@ package org.apache.tajo.master.scheduler; +import com.google.common.base.Objects; import org.apache.tajo.QueryId; public class QuerySchedulingInfo implements Comparable { @@ -68,4 +69,23 @@ public int compareTo(QuerySchedulingInfo o) { } return ret; } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + QuerySchedulingInfo other = (QuerySchedulingInfo) obj; + if (!this.getQueryId().equals(other.getQueryId())) + return false; + return true; + } + + @Override + public int hashCode() { + return Objects.hashCode(queryId, queue, user, priority, startTime); + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index fcb9978b88..f516003a1b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -73,7 +73,7 @@ public void handle(NodeResourceEvent event) { case ALLOCATE: { if (event.getResourceType() == NodeResourceEvent.ResourceType.TASK) { // allocate task resource - NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; + NodeResourceAllocateEvent allocateEvent = TUtil.checkTypeAndGet(event, NodeResourceAllocateEvent.class); BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { NodeResource resource = new NodeResource(request.getResource()); @@ -102,7 +102,7 @@ public void handle(NodeResourceEvent event) { break; } case DEALLOCATE: { - NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; + NodeResourceDeallocateEvent deallocateEvent = TUtil.checkTypeAndGet(event, NodeResourceDeallocateEvent.class); release(deallocateEvent.getResource()); if (deallocateEvent.getResourceType() == NodeResourceEvent.ResourceType.QUERY_MASTER) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 9e386d6754..ceee09a810 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -91,7 +91,7 @@ public void serviceStart() throws Exception { DefaultResourceCalculator calculator = new DefaultResourceCalculator(); int maxContainer = calculator.computeAvailableContainers(workerContext.getNodeResourceManager().getTotalResource(), NodeResources.createResource(tajoConf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY))); - this.queueingThreshold = (int) Math.ceil(maxContainer / 2); + this.queueingThreshold = (int) Math.floor(maxContainer * 0.5); LOG.info("Queueing threshold:" + queueingThreshold); updaterThread.start(); @@ -229,13 +229,8 @@ public void run() { break; } - if (!events.isEmpty()) { - // send current available resource; - lastResponse = sendHeartbeat(createResourceReport().build()); - } else { - // send ping; - lastResponse = sendHeartbeat(createResourceReport().build()); - } + // send current available resource; + lastResponse = sendHeartbeat(createResourceReport().build()); } else if (lastResponse.getCommand() == ResponseCommand.MEMBERSHIP) { // Membership changed diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index d6f00cffac..1e3615eaf1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -198,7 +198,7 @@ public void handle(TaskManagerEvent event) { workerContext.getNodeResourceManager().getDispatcher().getEventHandler() .handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); stopExecutionBlock(executionBlockContextMap.remove(executionBlockStopEvent.getExecutionBlockId()), - ((ExecutionBlockStopEvent) event).getCleanupList()); + executionBlockStopEvent.getCleanupList()); break; } case QUERY_STOP: { @@ -222,12 +222,15 @@ public void handle(TaskManagerEvent event) { LOG.error(errorEvent.getError().getMessage(), errorEvent.getError()); ExecutionBlockContext context = executionBlockContextMap.remove(errorEvent.getExecutionBlockId()); - if(context != null) { + if (context != null) { context.getSharedResource().releaseBroadcastCache(context.getExecutionBlockId()); getWorkerContext().getTaskHistoryWriter().flushTaskHistories(); context.stop(); } + break; } + default: + break; } } From 6d9a06b64cec6746395440477d61186791896ed0 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 15 Jul 2015 16:43:53 +0900 Subject: [PATCH 64/80] update minimum required cpu-cores --- .../src/main/java/org/apache/tajo/conf/TajoConf.java | 2 +- .../java/org/apache/tajo/worker/NodeStatusUpdater.java | 9 +++++---- 2 files changed, 6 insertions(+), 5 deletions(-) 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 c42046ab63..7939d30f19 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 @@ -173,7 +173,7 @@ public static enum ConfVars implements ConfigKey { // Tajo Worker Resources WORKER_RESOURCE_AVAILABLE_CPU_CORES("tajo.worker.resource.cpu-cores", - Runtime.getRuntime().availableProcessors(), Validators.min("1")), + Runtime.getRuntime().availableProcessors(), Validators.min("2")), // 1qm + 1task WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 1000, Validators.min("64")), WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1, Validators.min("1")), diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index ceee09a810..1d0d9893e6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -90,8 +90,9 @@ public void serviceStart() throws Exception { // if resource changed over than 50%, send reports DefaultResourceCalculator calculator = new DefaultResourceCalculator(); int maxContainer = calculator.computeAvailableContainers(workerContext.getNodeResourceManager().getTotalResource(), - NodeResources.createResource(tajoConf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY))); - this.queueingThreshold = (int) Math.floor(maxContainer * 0.5); + NodeResources.createResource(tajoConf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY), 1)); + + this.queueingThreshold = Math.max((int) Math.floor(maxContainer * 0.5), 1); LOG.info("Queueing threshold:" + queueingThreshold); updaterThread.start(); @@ -188,7 +189,6 @@ private int drain(Collection buffer, int numElements, long deadline = System.nanoTime() + unit.toNanos(timeout); int added = 0; while (added < numElements) { - added += heartBeatRequestQueue.drainTo(buffer, numElements - added); if (added < numElements) { // not enough elements immediately available; will have to wait NodeStatusEvent e = heartBeatRequestQueue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); if (e == null) { @@ -198,6 +198,7 @@ private int drain(Collection buffer, int numElements, added++; if (e.getType() == NodeStatusEvent.EventType.FLUSH_REPORTS) { + added += heartBeatRequestQueue.drainTo(buffer, numElements - added); break; } } @@ -224,7 +225,7 @@ public void run() { try { /* batch update to ResourceTracker */ - drain(events, Math.max(queueingThreshold, 1), nextHeartBeatInterval, TimeUnit.MILLISECONDS); + drain(events, queueingThreshold, nextHeartBeatInterval, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { break; } From b62b8d8c179e9f2c78a21ff3cb2109c31e049e9b Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 15 Jul 2015 17:54:44 +0900 Subject: [PATCH 65/80] remove unused resource allocation --- .../org/apache/tajo/querymaster/DefaultTaskScheduler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 40a5b3d7e6..790db6958b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -151,14 +151,14 @@ public void stop() { public void schedule() throws Exception{ try { - LinkedList taskRequests = createTaskRequest(); - if (remainingScheduledObjectNum() == 0) { // all task is done, wait for stopping message synchronized (schedulingThread) { schedulingThread.wait(500); } } else { + LinkedList taskRequests = createTaskRequest(); + if (taskRequests.size() == 0) { synchronized (schedulingThread) { schedulingThread.wait(schedulerDelay); From 3a80841f97aee3c553093a17a6181dccbf6f5b5d Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 16 Jul 2015 16:40:28 +0900 Subject: [PATCH 66/80] apply pb name rule --- .../org/apache/tajo/util/StringUtils.java | 2 +- tajo-core/pom.xml | 1 + .../engine/planner/PhysicalPlannerImpl.java | 24 +- .../tajo/engine/planner/enforce/Enforcer.java | 13 +- .../engine/planner/global/DataChannel.java | 3 +- .../engine/planner/global/MasterPlan.java | 11 +- .../builder/DistinctGroupbyBuilder.java | 6 +- .../engine/planner/physical/ScanExec.java | 10 +- .../apache/tajo/engine/query/TaskRequest.java | 4 +- .../tajo/engine/query/TaskRequestImpl.java | 14 +- .../tajo/master/QueryCoordinatorService.java | 12 +- .../apache/tajo/master/QueryInProgress.java | 20 +- .../org/apache/tajo/master/QueryManager.java | 13 +- .../master/event/StageShuffleReportEvent.java | 8 +- .../event/TaskAttemptStatusUpdateEvent.java | 2 +- .../event/TaskAttemptToSchedulerEvent.java | 10 +- .../master/event/TaskCompletionEvent.java | 2 +- .../master/event/TaskFatalErrorEvent.java | 2 +- .../tajo/master/event/TaskRequestEvent.java | 8 +- .../tajo/master/rm/TajoResourceManager.java | 5 - .../tajo/master/rm/TajoResourceTracker.java | 13 +- .../master/scheduler/SimpleScheduler.java | 24 +- .../scheduler/TajoResourceScheduler.java | 7 +- .../event/ResourceReserveSchedulerEvent.java | 16 +- .../querymaster/DefaultTaskScheduler.java | 48 ++- .../apache/tajo/querymaster/QueryMaster.java | 19 +- .../QueryMasterManagerService.java | 23 +- .../tajo/querymaster/QueryMasterTask.java | 8 +- .../tajo/querymaster/Repartitioner.java | 4 +- .../org/apache/tajo/querymaster/Stage.java | 16 +- .../org/apache/tajo/querymaster/Task.java | 4 +- .../apache/tajo/querymaster/TaskAttempt.java | 5 +- .../java/org/apache/tajo/session/Session.java | 2 +- .../tajo/util/history/HistoryReader.java | 2 +- .../tajo/worker/ExecutionBlockContext.java | 4 +- .../org/apache/tajo/worker/FetchImpl.java | 12 +- .../tajo/worker/NodeResourceManager.java | 6 +- .../apache/tajo/worker/NodeStatusUpdater.java | 22 +- .../tajo/worker/TajoWorkerManagerService.java | 9 +- .../java/org/apache/tajo/worker/Task.java | 4 +- .../org/apache/tajo/worker/TaskExecutor.java | 3 +- .../org/apache/tajo/worker/TaskHistory.java | 4 +- .../java/org/apache/tajo/worker/TaskImpl.java | 9 +- .../org/apache/tajo/worker/TaskManager.java | 14 +- .../worker/event/ExecutionBlockStopEvent.java | 8 +- .../event/NodeResourceAllocateEvent.java | 16 +- .../worker/event/QMResourceAllocateEvent.java | 8 +- .../tajo/worker/event/TaskStartEvent.java | 2 +- .../main/proto/QueryCoordinatorProtocol.proto | 62 +--- .../src/main/proto/QueryMasterProtocol.proto | 10 +- tajo-core/src/main/proto/ResourceProtos.proto | 311 ++++++++++++++++ .../main/proto/ResourceTrackerProtocol.proto | 33 +- .../src/main/proto/TajoWorkerProtocol.proto | 346 +----------------- .../planner/physical/TestBNLJoinExec.java | 2 +- .../physical/TestFullOuterHashJoinExec.java | 2 +- .../physical/TestFullOuterMergeJoinExec.java | 2 +- .../planner/physical/TestHashJoinExec.java | 2 +- .../physical/TestLeftOuterHashJoinExec.java | 2 +- .../planner/physical/TestMergeJoinExec.java | 2 +- .../planner/physical/TestPhysicalPlanner.java | 4 +- .../physical/TestRightOuterMergeJoinExec.java | 2 +- .../apache/tajo/master/TestRepartitioner.java | 11 +- .../master/scheduler/TestSimpleScheduler.java | 18 +- .../tajo/querymaster/TestKillQuery.java | 6 +- .../tajo/worker/MockExecutionBlock.java | 4 +- .../tajo/worker/MockNodeResourceManager.java | 16 +- .../tajo/worker/MockNodeStatusUpdater.java | 15 +- .../apache/tajo/worker/MockTaskExecutor.java | 9 +- .../apache/tajo/worker/MockTaskManager.java | 7 +- .../tajo/worker/TestNodeResourceManager.java | 38 +- .../tajo/worker/TestNodeStatusUpdater.java | 3 +- .../apache/tajo/worker/TestTaskExecutor.java | 18 +- .../apache/tajo/worker/TestTaskManager.java | 8 +- tajo-plan/src/main/proto/Plan.proto | 125 +++++++ 74 files changed, 772 insertions(+), 768 deletions(-) create mode 100644 tajo-core/src/main/proto/ResourceProtos.proto diff --git a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java index b8f3254ee9..018c62aeab 100644 --- a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java +++ b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java @@ -79,7 +79,7 @@ public static String formatTime(long timeDiff){ return buf.toString(); } /** - * or "ISO-8859-1" for ISO Latin 1 + * Check Seven-bit ASCII */ public static boolean isPureAscii(String v) { // get thread-safe encoder diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 6ad6956730..19c0e2ca4b 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -169,6 +169,7 @@ src/main/proto/QueryCoordinatorProtocol.proto src/main/proto/TajoWorkerProtocol.proto src/main/proto/InternalTypes.proto + src/main/proto/ResourceProtos.proto diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java index c6b9b4132b..377aebe22a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java @@ -40,11 +40,10 @@ import org.apache.tajo.engine.planner.physical.*; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.exception.InternalException; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.DistinctAggregationAlgorithm; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.SortSpecArray; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.DistinctAggregationAlgorithm; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.SortSpecArray; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.serder.LogicalNodeDeserializer; @@ -66,12 +65,13 @@ import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm; -import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty; -import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; -import static org.apache.tajo.ipc.TajoWorkerProtocol.GroupbyEnforce.GroupbyAlgorithm; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; -import static org.apache.tajo.ipc.TajoWorkerProtocol.SortEnforce; +import static org.apache.tajo.plan.serder.PlanProto.ColumnPartitionEnforcer.ColumnPartitionAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.EnforceProperty; +import static org.apache.tajo.plan.serder.PlanProto.EnforceProperty.EnforceType; +import static org.apache.tajo.plan.serder.PlanProto.GroupbyEnforce.GroupbyAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.SortedInputEnforce; +import static org.apache.tajo.plan.serder.PlanProto.SortEnforce; public class PhysicalPlannerImpl implements PhysicalPlanner { private static final Log LOG = LogFactory.getLog(PhysicalPlannerImpl.class); @@ -885,7 +885,7 @@ private boolean checkIfSortEquivalance(TaskAttemptContext ctx, ScanNode scanNode List property = enforcer.getEnforceProperties(EnforceType.SORTED_INPUT); if (property != null && property.size() > 0 && node.peek().getType() == NodeType.SORT) { SortNode sortNode = (SortNode) node.peek(); - TajoWorkerProtocol.SortedInputEnforce sortEnforcer = property.get(0).getSortedInput(); + SortedInputEnforce sortEnforcer = property.get(0).getSortedInput(); boolean condition = scanNode.getTableName().equals(sortEnforcer.getTableName()); SortSpec [] sortSpecs = LogicalNodeDeserializer.convertSortSpecs(sortEnforcer.getSortSpecsList()); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java index 8128390933..92ecadda62 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/enforce/Enforcer.java @@ -18,24 +18,21 @@ package org.apache.tajo.engine.planner.enforce; - import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.SortSpec; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.ProtoObject; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.DistinctAggregationAlgorithm; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.SortSpecArray; import org.apache.tajo.util.TUtil; import java.util.Collection; import java.util.List; import java.util.Map; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm; -import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; -import static org.apache.tajo.ipc.TajoWorkerProtocol.GroupbyEnforce.GroupbyAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.*; +import static org.apache.tajo.plan.serder.PlanProto.GroupbyEnforce.GroupbyAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.*; +import static org.apache.tajo.plan.serder.PlanProto.EnforceProperty.EnforceType; +import static org.apache.tajo.plan.serder.PlanProto.ColumnPartitionEnforcer.ColumnPartitionAlgorithm; public class Enforcer implements ProtoObject { Map> properties; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java index 3adc0a332a..e09684a9b1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/DataChannel.java @@ -25,8 +25,7 @@ import org.apache.tajo.catalog.SchemaUtil; import org.apache.tajo.util.StringUtils; -import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.apache.tajo.plan.serder.PlanProto.DataChannelProto; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; import static org.apache.tajo.plan.serder.PlanProto.TransmitType; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java index f8cd1e918a..80317b01ed 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java @@ -23,13 +23,14 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryId; +import org.apache.tajo.engine.planner.enforce.Enforcer; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.plan.LogicalPlan; +import org.apache.tajo.plan.serder.PlanProto.EnforceProperty; +import org.apache.tajo.plan.serder.PlanProto.ShuffleType; import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.util.graph.DirectedGraphVisitor; import org.apache.tajo.util.graph.SimpleDirectedGraph; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.TajoWorkerProtocol; import java.util.ArrayList; import java.util.HashMap; @@ -37,8 +38,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; - public class MasterPlan { private final QueryId queryId; private final QueryContext context; @@ -288,7 +287,7 @@ public String toString() { if (block.getEnforcer().getProperties().size() > 0) { sb.append("\n[Enforcers]\n"); int i = 0; - for (TajoWorkerProtocol.EnforceProperty enforce : block.getEnforcer().getProperties()) { + for (EnforceProperty enforce : block.getEnforcer().getProperties()) { sb.append(" ").append(i++).append(": "); sb.append(Enforcer.toString(enforce)); sb.append("\n"); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java index 8095458591..f181193593 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/builder/DistinctGroupbyBuilder.java @@ -30,9 +30,9 @@ import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.engine.planner.global.GlobalPlanner.GlobalPlanContext; import org.apache.tajo.engine.planner.global.MasterPlan; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.DistinctAggregationAlgorithm; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.SortSpecArray; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.DistinctAggregationAlgorithm; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.SortSpecArray; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.plan.PlanningException; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java index 86874ba3cc..5cca4c59bc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java @@ -21,7 +21,7 @@ import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.engine.planner.enforce.Enforcer; -import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.plan.serder.PlanProto.EnforceProperty; import org.apache.tajo.worker.TaskAttemptContext; import java.io.IOException; @@ -57,11 +57,11 @@ public boolean canBroadcast() { private boolean checkIfBroadcast() { Enforcer enforcer = context.getEnforcer(); - if (enforcer != null && enforcer.hasEnforceProperty(TajoWorkerProtocol.EnforceProperty.EnforceType.BROADCAST)) { - List properties = - enforcer.getEnforceProperties(TajoWorkerProtocol.EnforceProperty.EnforceType.BROADCAST); + if (enforcer != null && enforcer.hasEnforceProperty(EnforceProperty.EnforceType.BROADCAST)) { + List properties = + enforcer.getEnforceProperties(EnforceProperty.EnforceType.BROADCAST); - for (TajoWorkerProtocol.EnforceProperty property : properties) { + for (EnforceProperty property : properties) { if (getCanonicalName().equals(property.getBroadcast().getTableName())) { return true; } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java index 6429d4001b..98ccdb3c7e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java @@ -21,19 +21,19 @@ */ package org.apache.tajo.engine.query; +import org.apache.tajo.ResourceProtos.TaskRequestProto; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.ProtoObject; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.engine.planner.global.DataChannel; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.worker.FetchImpl; import java.util.List; -public interface TaskRequest extends ProtoObject { +public interface TaskRequest extends ProtoObject { String getQueryMasterHostAndPort(); TaskAttemptId getId(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java index afb6cc003d..6355046ff4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java @@ -22,9 +22,9 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.engine.planner.global.DataChannel; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProtoOrBuilder; +import org.apache.tajo.ResourceProtos.TaskRequestProto; +import org.apache.tajo.ResourceProtos.FetchProto; +import org.apache.tajo.ResourceProtos.TaskRequestProtoOrBuilder; import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.worker.FetchImpl; @@ -48,8 +48,8 @@ public class TaskRequestImpl implements TaskRequest { private Enforcer enforcer; private String queryMasterHostAndPort; - private TaskRequestProto proto = TajoWorkerProtocol.TaskRequestProto.getDefaultInstance(); - private TajoWorkerProtocol.TaskRequestProto.Builder builder = null; + private TaskRequestProto proto = TaskRequestProto.getDefaultInstance(); + private TaskRequestProto.Builder builder = null; private boolean viaProto = false; public TaskRequestImpl() { @@ -263,14 +263,14 @@ private void initFetches() { } TaskRequestProtoOrBuilder p = viaProto ? proto : builder; this.fetches = new ArrayList(); - for(TajoWorkerProtocol.FetchProto fetch : p.getFetchesList()) { + for(FetchProto fetch : p.getFetchesList()) { fetches.add(new FetchImpl(fetch)); } } private void maybeInitBuilder() { if (viaProto || builder == null) { - builder = TajoWorkerProtocol.TaskRequestProto.newBuilder(proto); + builder = TaskRequestProto.newBuilder(proto); } viaProto = true; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java index 8acc9d901b..9e508e4765 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java @@ -38,6 +38,8 @@ import java.net.InetSocketAddress; import java.util.Collection; +import static org.apache.tajo.ResourceProtos.*; + public class QueryCoordinatorService extends AbstractService { private final static Log LOG = LogFactory.getLog(QueryCoordinatorService.class); @@ -93,7 +95,7 @@ private class ProtocolServiceHandler implements QueryCoordinatorProtocolService. @Override public void heartbeat( RpcController controller, - TajoHeartbeat request, RpcCallback done) { + TajoHeartbeatRequest request, RpcCallback done) { if(LOG.isDebugEnabled()) { LOG.debug("Received QueryHeartbeat:" + new WorkerConnectionInfo(request.getConnectionInfo())); } @@ -113,17 +115,17 @@ public void heartbeat( } @Override - public void reserveNodeResources(RpcController controller, NodeResourceRequestProto request, - RpcCallback done) { + public void reserveNodeResources(RpcController controller, NodeResourceRequest request, + RpcCallback done) { Dispatcher dispatcher = context.getResourceManager().getRMContext().getDispatcher(); dispatcher.getEventHandler().handle(new ResourceReserveSchedulerEvent(request, done)); } @Override public void getAllWorkers(RpcController controller, PrimitiveProtos.NullProto request, - RpcCallback done) { + RpcCallback done) { - WorkerConnectionsProto.Builder builder = WorkerConnectionsProto.newBuilder(); + WorkerConnectionsResponse.Builder builder = WorkerConnectionsResponse.newBuilder(); Collection workers = context.getResourceManager().getRMContext().getWorkers().values(); for(Worker worker: workers) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java index 1c31989602..6265b01ca7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java @@ -22,17 +22,19 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.StringUtils; import org.apache.tajo.QueryId; +import org.apache.tajo.ResourceProtos.AllocationResourceProto; +import org.apache.tajo.ResourceProtos.QueryExecutionRequest; import org.apache.tajo.TajoProtos; import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; 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.QueryExecutionRequestProto; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.rm.TajoResourceManager; import org.apache.tajo.plan.logical.LogicalRootNode; -import org.apache.tajo.rpc.*; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.rpc.NettyClientBase; +import org.apache.tajo.rpc.RpcClientManager; +import org.apache.tajo.rpc.RpcConstants; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.session.Session; import org.apache.tajo.util.NetUtils; @@ -66,7 +68,7 @@ public class QueryInProgress { private QueryMasterProtocolService queryMasterRpcClient; - private QueryCoordinatorProtocol.AllocationResourceProto allocationResource; + private AllocationResourceProto allocationResource; private final Lock readLock; private final Lock writeLock; @@ -127,9 +129,11 @@ public void stopProgress() { /** * Connect to QueryMaster and allocate QM resource. - * If there is no available resource, It returns false + * + * @param allocation QM resource + * @return If there is no available resource, It returns false */ - protected boolean allocateToQueryMaster(QueryCoordinatorProtocol.AllocationResourceProto allocation) { + protected boolean allocateToQueryMaster(AllocationResourceProto allocation) { try { writeLock.lockInterruptibly(); } catch (Exception e) { @@ -197,7 +201,7 @@ public boolean submitToQueryMaster() { LOG.info("Call executeQuery to :" + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort() + "," + queryId); - QueryExecutionRequestProto.Builder builder = TajoWorkerProtocol.QueryExecutionRequestProto.newBuilder(); + QueryExecutionRequest.Builder builder = QueryExecutionRequest.newBuilder(); builder.setQueryId(queryId.getProto()) .setQueryContext(queryInfo.getQueryContext().getProto()) .setSession(session.getProto()) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java index 00dd1de672..d8c1ea1acd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java @@ -29,10 +29,13 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.QueryId; import org.apache.tajo.QueryIdFactory; +import org.apache.tajo.ResourceProtos; +import org.apache.tajo.ResourceProtos.AllocationResourceProto; +import org.apache.tajo.ResourceProtos.TajoHeartbeatRequest; +import org.apache.tajo.ResourceProtos.TajoHeartbeatResponse; import org.apache.tajo.TajoProtos; import org.apache.tajo.catalog.TableDesc; import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.scheduler.QuerySchedulingInfo; import org.apache.tajo.plan.logical.LogicalRootNode; @@ -184,7 +187,7 @@ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, Strin /** * Can start query or not */ - public boolean startQueryJob(QueryId queryId, QueryCoordinatorProtocol.AllocationResourceProto allocation) { + public boolean startQueryJob(QueryId queryId, AllocationResourceProto allocation) { if (submittedQueries.get(queryId).allocateToQueryMaster(allocation)) { QueryInProgress queryInProgress = submittedQueries.remove(queryId); @@ -282,8 +285,8 @@ public long getExecutedQuerySize() { return executedQuerySize.get(); } - public synchronized QueryCoordinatorProtocol.TajoHeartbeatResponse.ResponseCommand queryHeartbeat( - QueryCoordinatorProtocol.TajoHeartbeat queryHeartbeat) { + public synchronized TajoHeartbeatResponse.ResponseCommand queryHeartbeat( + TajoHeartbeatRequest queryHeartbeat) { QueryInProgress queryInProgress = getQueryInProgress(new QueryId(queryHeartbeat.getQueryId())); if(queryInProgress == null) { return null; @@ -295,7 +298,7 @@ public synchronized QueryCoordinatorProtocol.TajoHeartbeatResponse.ResponseComma return null; } - private QueryInfo makeQueryInfoFromHeartbeat(QueryCoordinatorProtocol.TajoHeartbeat queryHeartbeat) { + private QueryInfo makeQueryInfoFromHeartbeat(ResourceProtos.TajoHeartbeatRequest queryHeartbeat) { QueryInfo queryInfo = new QueryInfo(new QueryId(queryHeartbeat.getQueryId())); WorkerConnectionInfo connectionInfo = new WorkerConnectionInfo(queryHeartbeat.getConnectionInfo()); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageShuffleReportEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageShuffleReportEvent.java index 924fb59540..8a3dcb0177 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/StageShuffleReportEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageShuffleReportEvent.java @@ -19,20 +19,20 @@ package org.apache.tajo.master.event; import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.ResourceProtos.ExecutionBlockReport; /** * Event Class: From {@link org.apache.tajo.querymaster.QueryMasterManagerService} to Stage */ public class StageShuffleReportEvent extends StageEvent { - private TajoWorkerProtocol.ExecutionBlockReport report; + private ExecutionBlockReport report; - public StageShuffleReportEvent(ExecutionBlockId executionBlockId, TajoWorkerProtocol.ExecutionBlockReport report) { + public StageShuffleReportEvent(ExecutionBlockId executionBlockId, ExecutionBlockReport report) { super(executionBlockId, StageEventType.SQ_SHUFFLE_REPORT); this.report = report; } - public TajoWorkerProtocol.ExecutionBlockReport getReport() { + public ExecutionBlockReport getReport() { return report; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java index 8c5f016ccb..a9af288842 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java @@ -19,7 +19,7 @@ package org.apache.tajo.master.event; import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskStatusProto; +import org.apache.tajo.ResourceProtos.TaskStatusProto; public class TaskAttemptStatusUpdateEvent extends TaskAttemptEvent { private final TaskStatusProto status; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java index 6b939732da..1eee40d052 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java @@ -20,7 +20,7 @@ import com.google.protobuf.RpcCallback; import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.ResourceProtos.TaskRequestProto; import org.apache.tajo.querymaster.TaskAttempt; public class TaskAttemptToSchedulerEvent extends TaskSchedulerEvent { @@ -44,14 +44,14 @@ public TaskAttemptScheduleContext getContext() { public static class TaskAttemptScheduleContext { private String host; - private RpcCallback callback; + private RpcCallback callback; public TaskAttemptScheduleContext() { } public TaskAttemptScheduleContext(String host, - RpcCallback callback) { + RpcCallback callback) { this.host = host; this.callback = callback; } @@ -64,11 +64,11 @@ public void setHost(String host) { this.host = host; } - public RpcCallback getCallback() { + public RpcCallback getCallback() { return callback; } - public void setCallback(RpcCallback callback) { + public void setCallback(RpcCallback callback) { this.callback = callback; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java index 20204aaab1..66275b1f48 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java @@ -19,7 +19,7 @@ package org.apache.tajo.master.event; import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport; +import org.apache.tajo.ResourceProtos.TaskCompletionReport; public class TaskCompletionEvent extends TaskAttemptEvent { private TaskCompletionReport report; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java index 03888bdce7..d50fcb889a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java @@ -19,7 +19,7 @@ package org.apache.tajo.master.event; import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskFatalErrorReport; +import org.apache.tajo.ResourceProtos.TaskFatalErrorReport; public class TaskFatalErrorEvent extends TaskAttemptEvent { private final String message; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java index c686078f54..495eaf2154 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java @@ -20,7 +20,7 @@ import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ResourceProtos.AllocationResourceProto; import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType; public class TaskRequestEvent extends AbstractEvent { @@ -29,12 +29,12 @@ public enum TaskRequestEventType { TASK_REQ } - private final QueryCoordinatorProtocol.AllocationResourceProto responseProto; + private final AllocationResourceProto responseProto; private final ExecutionBlockId executionBlockId; private final int workerId; public TaskRequestEvent(int workerId, - QueryCoordinatorProtocol.AllocationResourceProto responseProto, + AllocationResourceProto responseProto, ExecutionBlockId executionBlockId) { super(TaskRequestEventType.TASK_REQ); this.workerId = workerId; @@ -46,7 +46,7 @@ public ExecutionBlockId getExecutionBlockId() { return executionBlockId; } - public QueryCoordinatorProtocol.AllocationResourceProto getResponseProto() { + public AllocationResourceProto getResponseProto() { return responseProto; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java index af3d711ed7..8946a0f5ce 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java @@ -72,11 +72,6 @@ public TajoResourceManager(TajoMaster.MasterContext masterContext) { this.masterContext = masterContext; } - @VisibleForTesting - public TajoResourceManager(TajoConf systemConf) { - super(TajoResourceManager.class.getSimpleName()); - } - @Override public void serviceInit(Configuration conf) throws Exception { this.systemConf = TUtil.checkTypeAndGet(conf, TajoConf.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index a9e5abc2bf..8fbf0c403c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -26,6 +26,7 @@ import org.apache.hadoop.service.AbstractService; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; +import org.apache.tajo.ipc.TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.scheduler.event.SchedulerEvent; import org.apache.tajo.master.scheduler.event.SchedulerEventType; @@ -36,7 +37,7 @@ import java.net.InetSocketAddress; -import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; +import static org.apache.tajo.ResourceProtos.*; /** * It receives pings that workers periodically send. The ping messages contains the worker resources and their statuses. @@ -109,7 +110,7 @@ public void serviceStop() throws Exception { super.serviceStop(); } - private static WorkerStatusEvent createStatusEvent(NodeHeartbeatRequestProto heartbeat) { + private static WorkerStatusEvent createStatusEvent(NodeHeartbeatRequest heartbeat) { return new WorkerStatusEvent( heartbeat.getWorkerId(), heartbeat.getRunningTasks(), @@ -121,10 +122,10 @@ private static WorkerStatusEvent createStatusEvent(NodeHeartbeatRequestProto hea @Override public void nodeHeartbeat( RpcController controller, - NodeHeartbeatRequestProto heartbeat, - RpcCallback done) { + NodeHeartbeatRequest heartbeat, + RpcCallback done) { - NodeHeartbeatResponseProto.Builder response = NodeHeartbeatResponseProto.newBuilder(); + NodeHeartbeatResponse.Builder response = NodeHeartbeatResponse.newBuilder(); ResponseCommand responseCommand = ResponseCommand.NORMAL; try { // get a workerId from the heartbeat @@ -198,7 +199,7 @@ public void nodeHeartbeat( } } - private Worker createWorkerResource(NodeHeartbeatRequestProto request) { + private Worker createWorkerResource(NodeHeartbeatRequest request) { return new Worker(rmContext, new NodeResource(request.getTotalResource()), new WorkerConnectionInfo(request.getConnectionInfo())); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index c3b3831f8f..0163d12745 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -27,7 +27,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.tajo.QueryId; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.master.QueryInfo; import org.apache.tajo.master.TajoMaster; import org.apache.tajo.master.cluster.WorkerConnectionInfo; @@ -45,12 +44,12 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.PriorityBlockingQueue; -import static org.apache.tajo.ipc.QueryCoordinatorProtocol.AllocationResourceProto; +import static org.apache.tajo.ResourceProtos.*; public class SimpleScheduler extends AbstractQueryScheduler { private static final Log LOG = LogFactory.getLog(SimpleScheduler.class); - private static final float PARALLEL_QUERY_LIMIT = 0.5f; + private static final float MAXIMUM_RUNNING_QM_RATE = 0.5f; private static final Comparator COMPARATOR = new SchedulingAlgorithms.FifoComparator(); private volatile boolean isStopped = false; @@ -134,7 +133,7 @@ public ResourceCalculator getResourceCalculator() { return resourceCalculator; } - private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceRequest(QueryInfo queryInfo) { + private NodeResourceRequest createQMResourceRequest(QueryInfo queryInfo) { NodeResource qmResource = getQMMinimumResourceCapability(); int containers = 1; @@ -151,12 +150,11 @@ private QueryCoordinatorProtocol.NodeResourceRequestProto createQMResourceReques if (idleNode.size() > containers * 3) break; } - QueryCoordinatorProtocol.NodeResourceRequestProto.Builder builder = - QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); + NodeResourceRequest.Builder builder = NodeResourceRequest.newBuilder(); builder.setQueryId(queryInfo.getQueryId().getProto()) .setCapacity(qmResource.getProto()) - .setType(QueryCoordinatorProtocol.ResourceType.QUERYMASTER) + .setType(ResourceType.QUERYMASTER) .setPriority(1) .setNumContainers(containers) .setRunningTasks(1) @@ -174,7 +172,7 @@ public int getNumClusterNodes() { @Override public List - reserve(QueryId queryId, QueryCoordinatorProtocol.NodeResourceRequestProto request) { + reserve(QueryId queryId, NodeResourceRequest request) { List reservedResources; NodeResource capacity = new NodeResource(request.getCapacity()); @@ -272,8 +270,7 @@ protected void reserveResource(ResourceReserveSchedulerEvent schedulerEvent) { List resources = reserve(new QueryId(schedulerEvent.getRequest().getQueryId()), schedulerEvent.getRequest()); - QueryCoordinatorProtocol.NodeResourceResponseProto.Builder response = - QueryCoordinatorProtocol.NodeResourceResponseProto.newBuilder(); + NodeResourceResponse.Builder response = NodeResourceResponse.newBuilder(); response.setQueryId(schedulerEvent.getRequest().getQueryId()); schedulerEvent.getCallBack().run(response.addAllResource(resources).build()); } @@ -337,8 +334,8 @@ public void run() { int maxAvailable = getResourceCalculator().computeAvailableContainers( getMaximumResourceCapability(), getQMMinimumResourceCapability()); - // check maximum parallel running queries. allow 50% parallel running - if (assignedQueryMasterMap.size() >= Math.floor(maxAvailable * PARALLEL_QUERY_LIMIT)) { + // check maximum parallel running QM. allow 50% parallel running + if (assignedQueryMasterMap.size() >= Math.floor(maxAvailable * MAXIMUM_RUNNING_QM_RATE)) { queryQueue.add(query); synchronized (this) { try { @@ -352,8 +349,7 @@ public void run() { } } else { QueryInfo queryInfo = getQueryInfo(query.getQueryId()); - List allocation = - reserve(query.getQueryId(), createQMResourceRequest(queryInfo)); + List allocation = reserve(query.getQueryId(), createQMResourceRequest(queryInfo)); if(allocation.size() == 0) { queryQueue.add(query); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java index d73f28aed9..c7c37c4e3b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/TajoResourceScheduler.java @@ -20,7 +20,8 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.QueryId; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ResourceProtos.NodeResourceRequest; +import org.apache.tajo.ResourceProtos.AllocationResourceProto; import org.apache.tajo.master.scheduler.event.SchedulerEvent; import org.apache.tajo.resource.NodeResource; @@ -68,7 +69,7 @@ public interface TajoResourceScheduler extends EventHandler { * such as memory, CPU cores, and disk slots. * @return the number of reserved resources. */ - List - reserve(QueryId queryId, QueryCoordinatorProtocol.NodeResourceRequestProto ask); + List + reserve(QueryId queryId, NodeResourceRequest ask); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java index 80b3b3913d..47ee53b25c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/event/ResourceReserveSchedulerEvent.java @@ -19,27 +19,27 @@ package org.apache.tajo.master.scheduler.event; import com.google.protobuf.RpcCallback; -import static org.apache.tajo.ipc.QueryCoordinatorProtocol.NodeResourceRequestProto; -import static org.apache.tajo.ipc.QueryCoordinatorProtocol.NodeResourceResponseProto; +import static org.apache.tajo.ResourceProtos.NodeResourceRequest; +import static org.apache.tajo.ResourceProtos.NodeResourceResponse; public class ResourceReserveSchedulerEvent extends SchedulerEvent { - private NodeResourceRequestProto request; + private NodeResourceRequest request; - private RpcCallback callBack; + private RpcCallback callBack; - public ResourceReserveSchedulerEvent(NodeResourceRequestProto request, - RpcCallback callback) { + public ResourceReserveSchedulerEvent(NodeResourceRequest request, + RpcCallback callback) { super(SchedulerEventType.RESOURCE_RESERVE); this.request = request; this.callBack = callback; } - public NodeResourceRequestProto getRequest() { + public NodeResourceRequest getRequest() { return request; } - public RpcCallback getCallBack() { + public RpcCallback getCallBack() { return callBack; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 790db6958b..34543e7e34 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -27,12 +27,14 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.RackResolver; import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.TaskRequest; import org.apache.tajo.engine.query.TaskRequestImpl; import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ipc.QueryCoordinatorProtocol.QueryCoordinatorProtocolService; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; @@ -56,7 +58,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import static org.apache.tajo.ResourceProtos.*; public class DefaultTaskScheduler extends AbstractTaskScheduler { private static final Log LOG = LogFactory.getLog(DefaultTaskScheduler.class); @@ -273,28 +275,25 @@ protected LinkedList createTaskRequest() throws Exception { context.getMasterContext().getQueryMasterContext().getWorkerContext().getServiceTracker(); NettyClientBase tmClient = RpcClientManager.getInstance(). getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); - QueryCoordinatorProtocol.QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); + QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); - CallFuture callBack = new CallFuture(); - QueryCoordinatorProtocol.NodeResourceRequestProto.Builder request = - QueryCoordinatorProtocol.NodeResourceRequestProto.newBuilder(); + CallFuture callBack = new CallFuture(); + NodeResourceRequest.Builder request = NodeResourceRequest.newBuilder(); request.setCapacity(NodeResources.createResource(minTaskMemory, isLeaf ? 1 : 0).getProto()) .setNumContainers(requestContainerNum) .setPriority(stage.getPriority()) .setQueryId(context.getMasterContext().getQueryId().getProto()) - .setType(isLeaf ? QueryCoordinatorProtocol.ResourceType.LEAF : - QueryCoordinatorProtocol.ResourceType.INTERMEDIATE) + .setType(isLeaf ? ResourceType.LEAF : ResourceType.INTERMEDIATE) .setUserId(context.getMasterContext().getQueryContext().getUser()) .setRunningTasks(stage.getTotalScheduledObjectsCount() - stage.getCompletedTaskCount()) .addAllCandidateNodes(candidateWorkers) .setQueue(context.getMasterContext().getQueryContext().get("queue", "default")); //TODO set queue masterClientService.reserveNodeResources(callBack.getController(), request.build(), callBack); - QueryCoordinatorProtocol.NodeResourceResponseProto - responseProto = callBack.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); + NodeResourceResponse response = callBack.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); - for (QueryCoordinatorProtocol.AllocationResourceProto proto : responseProto.getResourceList()) { - taskRequestEvents.add(new TaskRequestEvent(proto.getWorkerId(), proto, context.getBlockId())); + for (AllocationResourceProto resource : response.getResourceList()) { + taskRequestEvents.add(new TaskRequestEvent(resource.getWorkerId(), resource, context.getBlockId())); } return taskRequestEvents; @@ -851,9 +850,8 @@ public void assignToLeafTasks(LinkedList taskRequests) { } //TODO send batch request - TajoWorkerProtocol.BatchAllocationRequestProto.Builder - requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); - requestProto.addTaskRequest(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); + requestProto.addTaskRequest(TaskAllocationProto.newBuilder() .setResource(taskRequest.getResponseProto().getResource()) .setTaskRequest(taskAssign.getProto()).build()); @@ -864,17 +862,17 @@ public void assignToLeafTasks(LinkedList taskRequests) { if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); AsyncRpcClient tajoWorkerRpc = null; - CallFuture - callFuture = new CallFuture(); + CallFuture callFuture = new CallFuture(); try { tajoWorkerRpc = RpcClientManager.getInstance().getClient(addr, TajoWorkerProtocol.class, true); TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); - TajoWorkerProtocol.BatchAllocationResponseProto - responseProto = callFuture.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); + BatchAllocationResponse responseProto = + callFuture.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); + if (responseProto.getCancellationTaskCount() > 0) { - for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { + for (TaskAllocationProto proto : responseProto.getCancellationTaskList()) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); cancellation++; } @@ -957,17 +955,15 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { context.getMasterContext().getWorkerMap().get(taskRequest.getWorkerId()); //TODO send batch request - TajoWorkerProtocol.BatchAllocationRequestProto.Builder - requestProto = TajoWorkerProtocol.BatchAllocationRequestProto.newBuilder(); - requestProto.addTaskRequest(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); + requestProto.addTaskRequest(TaskAllocationProto.newBuilder() .setResource(taskRequest.getResponseProto().getResource()) .setTaskRequest(taskAssign.getProto()).build()); requestProto.setExecutionBlockId(attemptId.getTaskId().getExecutionBlockId().getProto()); context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId, connectionInfo)); - CallFuture - callFuture = new CallFuture(); + CallFuture callFuture = new CallFuture(); InetSocketAddress addr = stage.getAssignedWorkerMap().get(connectionInfo.getId()); if (addr == null) addr = new InetSocketAddress(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()); @@ -978,11 +974,11 @@ public void assignToNonLeafTasks(LinkedList taskRequests) { TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub(); tajoWorkerRpcClient.allocateTasks(callFuture.getController(), requestProto.build(), callFuture); - TajoWorkerProtocol.BatchAllocationResponseProto + BatchAllocationResponse responseProto = callFuture.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); if(responseProto.getCancellationTaskCount() > 0) { - for (TajoWorkerProtocol.TaskAllocationRequestProto proto : responseProto.getCancellationTaskList()) { + for (TaskAllocationProto proto : responseProto.getCancellationTaskList()) { cancel(task.getAttempt(new TaskAttemptId(proto.getTaskRequest().getId()))); cancellation++; } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java index 81a0a7ef33..e07b43fc9a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java @@ -30,15 +30,16 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.SystemClock; import org.apache.tajo.QueryId; + import org.apache.tajo.TajoProtos; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.global.GlobalPlanner; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol.QueryCoordinatorProtocolService; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeat; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeatResponse; -import org.apache.tajo.ipc.QueryCoordinatorProtocol.WorkerConnectionsProto; +import org.apache.tajo.ResourceProtos.TajoHeartbeatRequest; +import org.apache.tajo.ResourceProtos.TajoHeartbeatResponse; +import org.apache.tajo.ResourceProtos.WorkerConnectionsResponse; import org.apache.tajo.master.event.QueryStartEvent; import org.apache.tajo.master.event.QueryStopEvent; import org.apache.tajo.rpc.*; @@ -180,11 +181,11 @@ public List getAllWorker() { rpc = manager.getClient(serviceTracker.getUmbilicalAddress(), QueryCoordinatorProtocol.class, true); QueryCoordinatorProtocolService masterService = rpc.getStub(); - CallFuture callBack = new CallFuture(); + CallFuture callBack = new CallFuture(); masterService.getAllWorkers(callBack.getController(), PrimitiveProtos.NullProto.getDefaultInstance(), callBack); - WorkerConnectionsProto connectionsProto = + WorkerConnectionsResponse connectionsProto = callBack.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); return connectionsProto.getWorkerList(); } catch (Exception e) { @@ -285,7 +286,7 @@ public void stopQuery(QueryId queryId) { finishedQueryMasterTasksCache.put(queryId, queryMasterTask); - TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(queryMasterTask); + TajoHeartbeatRequest queryHeartbeat = buildTajoHeartBeat(queryMasterTask); CallFuture future = new CallFuture(); NettyClientBase tmClient; @@ -323,8 +324,8 @@ public void stopQuery(QueryId queryId) { } } - private TajoHeartbeat buildTajoHeartBeat(QueryMasterTask queryMasterTask) { - TajoHeartbeat.Builder builder = TajoHeartbeat.newBuilder(); + private TajoHeartbeatRequest buildTajoHeartBeat(QueryMasterTask queryMasterTask) { + TajoHeartbeatRequest.Builder builder = TajoHeartbeatRequest.newBuilder(); builder.setConnectionInfo(workerContext.getConnectionInfo().getProto()); builder.setQueryId(queryMasterTask.getQueryId().getProto()); @@ -390,7 +391,7 @@ public void run() { QueryCoordinatorProtocol.class, true); QueryCoordinatorProtocolService masterClientService = tmClient.getStub(); - TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(eachTask); + TajoHeartbeatRequest queryHeartbeat = buildTajoHeartBeat(eachTask); masterClientService.heartbeat(null, queryHeartbeat, NullCallback.get()); } catch (Throwable t) { t.printStackTrace(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java index 25c7261035..62216aab10 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java @@ -28,9 +28,7 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryMasterProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.event.*; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.serder.PlanProto; @@ -46,6 +44,8 @@ import java.net.InetSocketAddress; +import static org.apache.tajo.ResourceProtos.*; + public class QueryMasterManagerService extends CompositeService implements QueryMasterProtocol.QueryMasterProtocolService.Interface { private static final Log LOG = LogFactory.getLog(QueryMasterManagerService.class.getName()); @@ -112,7 +112,7 @@ public InetSocketAddress getBindAddr() { } @Override - public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatusProto request, + public void statusUpdate(RpcController controller, TaskStatusProto request, RpcCallback done) { QueryId queryId = new QueryId(request.getId().getTaskId().getExecutionBlockId().getQueryId()); TaskAttemptId attemptId = new TaskAttemptId(request.getId()); @@ -148,7 +148,7 @@ public void ping(RpcController controller, } @Override - public void fatalError(RpcController controller, TajoWorkerProtocol.TaskFatalErrorReport report, + public void fatalError(RpcController controller, TaskFatalErrorReport report, RpcCallback done) { QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask( new QueryId(report.getId().getTaskId().getExecutionBlockId().getQueryId())); @@ -161,7 +161,7 @@ public void fatalError(RpcController controller, TajoWorkerProtocol.TaskFatalErr } @Override - public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionReport report, + public void done(RpcController controller, TaskCompletionReport report, RpcCallback done) { QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask( new QueryId(report.getId().getTaskId().getExecutionBlockId().getQueryId())); @@ -173,7 +173,7 @@ public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionRepo @Override public void doneExecutionBlock( - RpcController controller, TajoWorkerProtocol.ExecutionBlockReport request, + RpcController controller, ExecutionBlockReport request, RpcCallback done) { QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(new QueryId(request.getEbId().getQueryId())); if (queryMasterTask != null) { @@ -185,8 +185,8 @@ public void doneExecutionBlock( @Override public void getExecutionBlockContext(RpcController controller, - TajoWorkerProtocol.ExecutionBlockContextRequestProto request, - RpcCallback done) { + ExecutionBlockContextRequest request, + RpcCallback done) { QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask( new QueryId(request.getExecutionBlockId().getQueryId())); @@ -197,8 +197,7 @@ public void getExecutionBlockContext(RpcController controller, // first request with starting ExecutionBlock PlanProto.ShuffleType shuffleType = stage.getDataChannel().getShuffleType(); - TajoWorkerProtocol.ExecutionBlockContextProto.Builder - ebRequestProto = TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); + ExecutionBlockContextResponse.Builder ebRequestProto = ExecutionBlockContextResponse.newBuilder(); ebRequestProto.setExecutionBlockId(request.getExecutionBlockId()) .setQueryContext(stage.getContext().getQueryContext().getProto()) .setQueryOutputPath(stage.getContext().getStagingDir().toString()) @@ -229,7 +228,7 @@ public void killQuery(RpcController controller, TajoIdProtos.QueryIdProto reques @Override public void executeQuery(RpcController controller, - TajoWorkerProtocol.QueryExecutionRequestProto request, + QueryExecutionRequest request, RpcCallback done) { workerContext.getWorkerSystemMetrics().counter("querymaster", "numQuery").inc(); @@ -245,7 +244,7 @@ public void executeQuery(RpcController controller, @Override public void allocateQueryMaster(RpcController controller, - QueryCoordinatorProtocol.AllocationResourceProto request, + AllocationResourceProto request, RpcCallback done) { CallFuture callFuture = new CallFuture(); workerContext.getNodeResourceManager().handle(new QMResourceAllocateEvent(request, callFuture)); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index 1fcc22eefd..bec16857be 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -70,6 +70,7 @@ import java.util.concurrent.atomic.AtomicLong; import static org.apache.tajo.TajoProtos.QueryState; +import static org.apache.tajo.ResourceProtos.*; public class QueryMasterTask extends CompositeService { private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName()); @@ -106,8 +107,7 @@ public class QueryMasterTask extends CompositeService { private NodeResource allocation; - private final List diagnostics = - new ArrayList(); + private final List diagnostics = new ArrayList(); private final ConcurrentMap workerMap = Maps.newConcurrentMap(); @@ -204,7 +204,7 @@ public void run() { LOG.info("Stopped QueryMasterTask:" + queryId); } - public void handleTaskFailed(TajoWorkerProtocol.TaskFatalErrorReport report) { + public void handleTaskFailed(TaskFatalErrorReport report) { synchronized(diagnostics) { if (diagnostics.size() < 10) { diagnostics.add(report); @@ -214,7 +214,7 @@ public void handleTaskFailed(TajoWorkerProtocol.TaskFatalErrorReport report) { getEventHandler().handle(new TaskFatalErrorEvent(report)); } - public Collection getDiagnostics() { + public Collection getDiagnostics() { synchronized(diagnostics) { return Collections.unmodifiableCollection(diagnostics); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java index f30fb642c1..50b84d4750 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java @@ -40,8 +40,8 @@ import org.apache.tajo.engine.planner.global.rewriter.rules.GlobalPlanRewriteUtil; import org.apache.tajo.engine.utils.TupleUtil; import org.apache.tajo.exception.InternalException; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage; -import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage; +import org.apache.tajo.plan.serder.PlanProto.EnforceProperty; import org.apache.tajo.querymaster.Task.IntermediateEntry; import org.apache.tajo.plan.logical.SortNode.SortPurpose; import org.apache.tajo.plan.util.PlannerUtil; diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java index f1d0d1622a..50b45a8541 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java @@ -41,9 +41,8 @@ import org.apache.tajo.engine.planner.global.ExecutionBlock; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage; -import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty; -import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; +import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage; +import org.apache.tajo.plan.serder.PlanProto.EnforceProperty; import org.apache.tajo.master.TaskState; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; @@ -75,6 +74,7 @@ import static org.apache.tajo.conf.TajoConf.ConfVars; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; +import static org.apache.tajo.ResourceProtos.*; /** @@ -681,7 +681,7 @@ protected Map getAssignedWorkerMap() { return workerMap; } - private void sendStopExecutionBlockEvent(final TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto) { + private void sendStopExecutionBlockEvent(final StopExecutionBlockRequest requestProto) { for (final InetSocketAddress worker : getAssignedWorkerMap().values()) { getContext().getQueryMasterContext().getEventExecutor().submit(new Runnable() { @@ -716,10 +716,8 @@ protected void stopExecutionBlock() { } } - TajoWorkerProtocol.StopExecutionBlockRequestProto.Builder - stopRequest = TajoWorkerProtocol.StopExecutionBlockRequestProto.newBuilder(); - TajoWorkerProtocol.ExecutionBlockListProto.Builder - cleanupList = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder(); + StopExecutionBlockRequest.Builder stopRequest = StopExecutionBlockRequest.newBuilder(); + ExecutionBlockListProto.Builder cleanupList = ExecutionBlockListProto.newBuilder(); cleanupList.addAllExecutionBlockId(Lists.newArrayList(ebIds)); stopRequest.setCleanupList(cleanupList.build()); @@ -1222,7 +1220,7 @@ protected void stopFinalization() { private void finalizeShuffleReport(StageShuffleReportEvent event, ShuffleType type) { if(!checkIfNeedFinalizing(type)) return; - TajoWorkerProtocol.ExecutionBlockReport report = event.getReport(); + ExecutionBlockReport report = event.getReport(); if (!report.getReportSuccess()) { stopFinalization(); diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java index a396ad0451..a586e4b377 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java @@ -34,8 +34,6 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.ipc.TajoWorkerProtocol.FailureIntermediateProto; -import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto; import org.apache.tajo.master.TaskState; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; @@ -58,7 +56,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput; +import static org.apache.tajo.ResourceProtos.*; public class Task implements EventHandler { /** Class Logger */ diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java index 5e68711506..6c48d3b202 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java @@ -27,7 +27,8 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport; +import org.apache.tajo.ResourceProtos.TaskCompletionReport; +import org.apache.tajo.ResourceProtos.ShuffleFileOutput; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext; @@ -42,8 +43,6 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput; - public class TaskAttempt implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskAttempt.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/session/Session.java b/tajo-core/src/main/java/org/apache/tajo/session/Session.java index ea7d29a373..0d067dcfef 100644 --- a/tajo-core/src/main/java/org/apache/tajo/session/Session.java +++ b/tajo-core/src/main/java/org/apache/tajo/session/Session.java @@ -32,7 +32,7 @@ import java.util.HashMap; import java.util.Map; -import static org.apache.tajo.ipc.TajoWorkerProtocol.SessionProto; +import static org.apache.tajo.ResourceProtos.SessionProto; public class Session implements SessionConstants, ProtoObject, Cloneable { private static final Log LOG = LogFactory.getLog(Session.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java index a8737c97ac..2acd5f6a2d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java @@ -27,7 +27,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.TajoWorkerProtocol.TaskHistoryProto; +import org.apache.tajo.ResourceProtos.TaskHistoryProto; import org.apache.tajo.master.QueryInfo; import org.apache.tajo.util.Bytes; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 71129fd6ee..57bedd28da 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -51,8 +51,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.tajo.ResourceProtos.*; import static org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.Interface; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; public class ExecutionBlockContext { /** class logger */ @@ -93,7 +93,7 @@ public class ExecutionBlockContext { private final Map taskHistories = Maps.newConcurrentMap(); - public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, ExecutionBlockContextProto request, + public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, ExecutionBlockContextResponse request, AsyncRpcClient queryMasterClient) throws IOException { this.executionBlockId = new ExecutionBlockId(request.getExecutionBlockId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java index 89c34046d7..07a9ba6ac6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java @@ -22,10 +22,10 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.ResourceProtos.FetchProto; import org.apache.tajo.common.ProtoObject; -import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.querymaster.Task; import org.apache.tajo.querymaster.Repartitioner; +import org.apache.tajo.querymaster.Task; import org.apache.tajo.util.TUtil; import java.net.URI; @@ -37,7 +37,7 @@ /** * FetchImpl information to indicate the locations of intermediate data. */ -public class FetchImpl implements ProtoObject, Cloneable { +public class FetchImpl implements ProtoObject, Cloneable { private Task.PullHost host; // The pull server host information private ShuffleType type; // hash or range partition method. private ExecutionBlockId executionBlockId; // The executionBlock id @@ -57,7 +57,7 @@ public FetchImpl() { attemptIds = new ArrayList(); } - public FetchImpl(TajoWorkerProtocol.FetchProto proto) { + public FetchImpl(FetchProto proto) { this(new Task.PullHost(proto.getHost(), proto.getPort()), proto.getType(), new ExecutionBlockId(proto.getExecutionBlockId()), @@ -112,8 +112,8 @@ public int hashCode() { } @Override - public TajoWorkerProtocol.FetchProto getProto() { - TajoWorkerProtocol.FetchProto.Builder builder = TajoWorkerProtocol.FetchProto.newBuilder(); + public FetchProto getProto() { + FetchProto.Builder builder = FetchProto.newBuilder(); builder.setHost(host.getHost()); builder.setPort(host.getPort()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index f516003a1b..0580ebce4c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -35,7 +35,7 @@ import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.apache.tajo.ResourceProtos.*; public class NodeResourceManager extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(NodeResourceManager.class); @@ -74,8 +74,8 @@ public void handle(NodeResourceEvent event) { if (event.getResourceType() == NodeResourceEvent.ResourceType.TASK) { // allocate task resource NodeResourceAllocateEvent allocateEvent = TUtil.checkTypeAndGet(event, NodeResourceAllocateEvent.class); - BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); - for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { + BatchAllocationResponse.Builder response = BatchAllocationResponse.newBuilder(); + for (TaskAllocationProto request : allocateEvent.getRequest().getTaskRequestList()) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { //send task start event to TaskExecutor diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 1d0d9893e6..5d91cc618b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -46,7 +46,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; +import static org.apache.tajo.ResourceProtos.*; /** * It periodically sends heartbeat to {@link org.apache.tajo.master.rm.TajoResourceTracker} via asynchronous rpc. @@ -64,7 +64,7 @@ public class NodeStatusUpdater extends AbstractService implements EventHandler callBack = new CallFuture(); + CallFuture callBack = new CallFuture(); resourceTracker.nodeHeartbeat(callBack.getController(), requestProto, callBack); response = callBack.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); @@ -209,7 +209,7 @@ private int drain(Collection buffer, int numElements, /* Node sends a heartbeats with its resource and status periodically to master. */ @Override public void run() { - NodeHeartbeatResponseProto lastResponse = null; + NodeHeartbeatResponse lastResponse = null; while (!isStopped && !Thread.interrupted()) { try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index 0eda37e5fe..7752211a07 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -25,6 +25,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; import org.apache.tajo.QueryId; +import org.apache.tajo.ResourceProtos.BatchAllocationRequest; +import org.apache.tajo.ResourceProtos.BatchAllocationResponse; +import org.apache.tajo.ResourceProtos.StopExecutionBlockRequest; import org.apache.tajo.TajoIdProtos; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.conf.TajoConf; @@ -104,8 +107,8 @@ public void ping(RpcController controller, @Override public void allocateTasks(RpcController controller, - TajoWorkerProtocol.BatchAllocationRequestProto request, - RpcCallback done) { + BatchAllocationRequest request, + RpcCallback done) { workerContext.getWorkerSystemMetrics().counter("query", "allocationRequestNum").inc(); workerContext.getNodeResourceManager().getDispatcher(). getEventHandler().handle(new NodeResourceAllocateEvent(request, done)); @@ -113,7 +116,7 @@ public void allocateTasks(RpcController controller, @Override public void stopExecutionBlock(RpcController controller, - TajoWorkerProtocol.StopExecutionBlockRequestProto requestProto, + StopExecutionBlockRequest requestProto, RpcCallback done) { try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index 32d3f0cae0..66216ee4b7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -18,7 +18,7 @@ package org.apache.tajo.worker; -import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.ResourceProtos.TaskStatusProto; import java.io.IOException; import java.util.List; @@ -49,7 +49,7 @@ public interface Task { ExecutionBlockContext getExecutionBlockContext(); - TajoWorkerProtocol.TaskStatusProto getReport(); + TaskStatusProto getReport(); TaskHistory createTaskHistory(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 90bb6bbcd2..5e1ccc1e5a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -35,6 +35,7 @@ import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; import org.apache.tajo.worker.event.NodeResourceEvent; import org.apache.tajo.worker.event.TaskStartEvent; +import org.apache.tajo.ResourceProtos.TaskRequestProto; import java.io.IOException; import java.util.Map; @@ -144,7 +145,7 @@ protected ExecutorService getFetcherExecutor() { protected Task createTask(ExecutionBlockContext executionBlockContext, - TajoWorkerProtocol.TaskRequestProto taskRequest) throws IOException { + TaskRequestProto taskRequest) throws IOException { Task task = null; TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); if (executionBlockContext.getTasks().containsKey(taskAttemptId)) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java index c2432ebaff..52b0d0bd4b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskHistory.java @@ -29,8 +29,8 @@ import java.util.List; import static org.apache.tajo.TajoProtos.TaskAttemptState; -import static org.apache.tajo.ipc.TajoWorkerProtocol.FetcherHistoryProto; -import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskHistoryProto; +import static org.apache.tajo.ResourceProtos.FetcherHistoryProto; +import static org.apache.tajo.ResourceProtos.TaskHistoryProto; /** * The history class for Task processing. diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java index 92d7932f1e..4c1090f75c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -37,15 +37,17 @@ import org.apache.tajo.catalog.TableDesc; import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.planner.physical.PhysicalExec; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequest; import org.apache.tajo.ipc.QueryMasterProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol.*; -import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto.ShuffleType; +import org.apache.tajo.plan.serder.PlanProto.EnforceProperty; +import org.apache.tajo.plan.serder.PlanProto.EnforceProperty.EnforceType; import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.serder.LogicalNodeDeserializer; @@ -65,8 +67,7 @@ import java.util.Map.Entry; import java.util.concurrent.ExecutorService; -import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; -import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; +import static org.apache.tajo.ResourceProtos.*; public class TaskImpl implements Task { private static final Log LOG = LogFactory.getLog(TaskImpl.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index 1e3615eaf1..f518fd3950 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -30,7 +30,6 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; import org.apache.tajo.ipc.QueryMasterProtocol; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.rpc.AsyncRpcClient; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.rpc.RpcClientManager; @@ -45,8 +44,9 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockContextProto; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ExecutionBlockContextRequestProto; +import static org.apache.tajo.ResourceProtos.ExecutionBlockListProto; +import static org.apache.tajo.ResourceProtos.ExecutionBlockContextRequest; +import static org.apache.tajo.ResourceProtos.ExecutionBlockContextResponse; /** * A TaskManager is responsible for managing executionBlock resource and tasks. @@ -115,16 +115,16 @@ protected ExecutionBlockContext createExecutionBlock(ExecutionBlockId executionB AsyncRpcClient client = null; try { InetSocketAddress address = NetUtils.createSocketAddr(queryMasterHostAndPort); - ExecutionBlockContextRequestProto.Builder request = ExecutionBlockContextRequestProto.newBuilder(); + ExecutionBlockContextRequest.Builder request = ExecutionBlockContextRequest.newBuilder(); request.setExecutionBlockId(executionBlockId.getProto()) .setWorker(getWorkerContext().getConnectionInfo().getProto()); client = RpcClientManager.getInstance().newClient(address, QueryMasterProtocol.class, true); QueryMasterProtocol.QueryMasterProtocolService.Interface stub = client.getStub(); - CallFuture callback = new CallFuture(); + CallFuture callback = new CallFuture(); stub.getExecutionBlockContext(callback.getController(), request.build(), callback); - ExecutionBlockContextProto contextProto = + ExecutionBlockContextResponse contextProto = callback.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); ExecutionBlockContext context = new ExecutionBlockContext(getWorkerContext(), contextProto, client); @@ -138,7 +138,7 @@ protected ExecutionBlockContext createExecutionBlock(ExecutionBlockId executionB } protected void stopExecutionBlock(ExecutionBlockContext context, - TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + ExecutionBlockListProto cleanupList) { if (context != null) { try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java index 75ccecab9c..a1dfe508d5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/ExecutionBlockStopEvent.java @@ -20,20 +20,20 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.TajoIdProtos; -import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.ResourceProtos.ExecutionBlockListProto; public class ExecutionBlockStopEvent extends TaskManagerEvent { - private TajoWorkerProtocol.ExecutionBlockListProto cleanupList; + private ExecutionBlockListProto cleanupList; private ExecutionBlockId executionBlockId; public ExecutionBlockStopEvent(TajoIdProtos.ExecutionBlockIdProto executionBlockId, - TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + ExecutionBlockListProto cleanupList) { super(EventType.EB_STOP); this.cleanupList = cleanupList; this.executionBlockId = new ExecutionBlockId(executionBlockId); } - public TajoWorkerProtocol.ExecutionBlockListProto getCleanupList() { + public ExecutionBlockListProto getCleanupList() { return cleanupList; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java index 284db0e8dd..0ee08365d5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -21,26 +21,26 @@ import com.google.protobuf.RpcCallback; -import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; -import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; +import static org.apache.tajo.ResourceProtos.BatchAllocationRequest; +import static org.apache.tajo.ResourceProtos.BatchAllocationResponse; public class NodeResourceAllocateEvent extends NodeResourceEvent { - private BatchAllocationRequestProto request; - private RpcCallback callback; + private BatchAllocationRequest request; + private RpcCallback callback; - public NodeResourceAllocateEvent(BatchAllocationRequestProto request, - RpcCallback callback) { + public NodeResourceAllocateEvent(BatchAllocationRequest request, + RpcCallback callback) { super(EventType.ALLOCATE, ResourceType.TASK); this.callback = callback; this.request = request; } - public BatchAllocationRequestProto getRequest() { + public BatchAllocationRequest getRequest() { return request; } - public RpcCallback getCallback() { + public RpcCallback getCallback() { return callback; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.java index fd6dfdf795..4422d4d2f9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/QMResourceAllocateEvent.java @@ -20,22 +20,22 @@ import com.google.protobuf.RpcCallback; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ResourceProtos.AllocationResourceProto; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; public class QMResourceAllocateEvent extends NodeResourceEvent { - private QueryCoordinatorProtocol.AllocationResourceProto request; + private AllocationResourceProto request; private RpcCallback callback; - public QMResourceAllocateEvent(QueryCoordinatorProtocol.AllocationResourceProto request, + public QMResourceAllocateEvent(AllocationResourceProto request, RpcCallback callback) { super(EventType.ALLOCATE, ResourceType.QUERY_MASTER); this.callback = callback; this.request = request; } - public QueryCoordinatorProtocol.AllocationResourceProto getRequest() { + public AllocationResourceProto getRequest() { return request; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java index 112afd1066..1fb0c49c4c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java @@ -22,7 +22,7 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.resource.NodeResource; -import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; +import static org.apache.tajo.ResourceProtos.TaskRequestProto; public class TaskStartEvent extends TaskManagerEvent { diff --git a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto index ee215659ef..be04dc44c3 100644 --- a/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto +++ b/tajo-core/src/main/proto/QueryCoordinatorProtocol.proto @@ -23,67 +23,13 @@ option java_outer_classname = "QueryCoordinatorProtocol"; option java_generic_services = true; option java_generate_equals_and_hash = true; -import "yarn_protos.proto"; -import "tajo_protos.proto"; -import "TajoIdProtos.proto"; -import "CatalogProtos.proto"; import "PrimitiveProtos.proto"; +import "ResourceProtos.proto"; package hadoop.yarn; -//deplecated -message TajoHeartbeat { - required WorkerConnectionInfoProto connectionInfo = 1; - optional QueryIdProto queryId = 2; - optional QueryState state = 3; - optional TableDescProto resultDesc = 4; - optional string statusMessage = 5; - optional float queryProgress = 6; -} -//deplecated -message TajoHeartbeatResponse { - message ResponseCommand { - required string command = 1; - repeated string params = 2; - } - required BoolProto heartbeatResult = 1; - optional ResponseCommand responseCommand = 3; -} - -enum ResourceType { - LEAF = 1; - INTERMEDIATE = 2; - QUERYMASTER = 3; -} - -message WorkerConnectionsProto { - repeated WorkerConnectionInfoProto worker = 1; -} - -message AllocationResourceProto { - required int32 workerId = 1; - required NodeResourceProto resource = 2; -} - -message NodeResourceRequestProto { - optional string queue = 1; - required string userId = 2; - required ResourceType type = 3; - required int32 priority = 4; - required QueryIdProto queryId = 5; - required int32 numContainers = 6; - required NodeResourceProto capacity = 7; - required int32 runningTasks = 8; - repeated int32 candidateNodes = 9; -} - -message NodeResourceResponseProto { - required QueryIdProto queryId = 1; - repeated AllocationResourceProto resource = 2; -} - service QueryCoordinatorProtocolService { - rpc heartbeat(TajoHeartbeat) returns (TajoHeartbeatResponse); - rpc reserveNodeResources(NodeResourceRequestProto) returns (NodeResourceResponseProto); - rpc getAllWorkers(NullProto) returns (WorkerConnectionsProto); + rpc heartbeat(TajoHeartbeatRequest) returns (TajoHeartbeatResponse); + rpc reserveNodeResources(NodeResourceRequest) returns (NodeResourceResponse); + rpc getAllWorkers(NullProto) returns (WorkerConnectionsResponse); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto index 33372b1380..f783f06642 100644 --- a/tajo-core/src/main/proto/QueryMasterProtocol.proto +++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto @@ -21,13 +21,9 @@ 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"; -import "TajoWorkerProtocol.proto"; -import "QueryCoordinatorProtocol.proto"; +import "ResourceProtos.proto"; package hadoop.yarn; @@ -38,10 +34,10 @@ service QueryMasterProtocolService { rpc fatalError(TaskFatalErrorReport) returns (NullProto); rpc done (TaskCompletionReport) returns (NullProto); rpc doneExecutionBlock(ExecutionBlockReport) returns (NullProto); - rpc getExecutionBlockContext(ExecutionBlockContextRequestProto) returns (ExecutionBlockContextProto); + rpc getExecutionBlockContext(ExecutionBlockContextRequest) returns (ExecutionBlockContextResponse); //from TajoMaster's QueryJobManager rpc killQuery(QueryIdProto) returns (NullProto); - rpc executeQuery(QueryExecutionRequestProto) returns (NullProto); + rpc executeQuery(QueryExecutionRequest) returns (NullProto); rpc allocateQueryMaster(AllocationResourceProto) returns (BoolProto); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/ResourceProtos.proto b/tajo-core/src/main/proto/ResourceProtos.proto new file mode 100644 index 0000000000..97bf05e89e --- /dev/null +++ b/tajo-core/src/main/proto/ResourceProtos.proto @@ -0,0 +1,311 @@ +/** + * 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"; +option java_outer_classname = "ResourceProtos"; +option java_generic_services = false; +option java_generate_equals_and_hash = true; + +import "tajo_protos.proto"; +import "TajoIdProtos.proto"; +import "CatalogProtos.proto"; +import "PrimitiveProtos.proto"; +import "Plan.proto"; + +enum ResponseCommand { + NORMAL = 1; //ping + MEMBERSHIP = 2; // request membership to worker node + ABORT_QUERY = 3; //query master failure + SHUTDOWN = 4; // black list +} + +//TODO add node health information +message NodeStatusProto { +} + +enum ResourceType { + LEAF = 1; + INTERMEDIATE = 2; + QUERYMASTER = 3; +} + +message AllocationResourceProto { + required int32 workerId = 1; + required NodeResourceProto resource = 2; +} + + +message ExecutionBlockListProto { + repeated ExecutionBlockIdProto executionBlockId = 1; +} + +message TaskAllocationProto { + required TaskRequestProto taskRequest = 1; + required NodeResourceProto resource = 2; +} + +message TaskRequestProto { + required string queryMasterHostAndPort = 1; + required TaskAttemptIdProto id = 2; + repeated FragmentProto fragments = 3; + required string outputTable = 4; + required bool clusteredOutput = 5; + required LogicalNodeTree plan = 6; + optional bool interQuery = 7 [default = false]; + repeated FetchProto fetches = 8; + optional KeyValueSetProto queryContext = 9; + optional DataChannelProto dataChannel = 10; + optional EnforcerProto enforcer = 11; +} + +message FetchProto { + required string host = 1; + required int32 port = 2; + required ShuffleType type = 3; + required ExecutionBlockIdProto executionBlockId = 4; + required int32 partitionId = 5; + required string name = 6; + optional string rangeParams = 7; + optional bool hasNext = 8 [default = false]; + + //repeated part + repeated int32 taskId = 9 [packed=true]; + repeated int32 attemptId = 10 [packed=true]; + + optional int64 offset = 11; + optional int64 length = 12; +} + +message TaskStatusProto { + required TaskAttemptIdProto id = 1; + required string workerName = 2; + required float progress = 3; + required TaskAttemptState state = 4; + optional StatSetProto stats = 5; + optional TableStatsProto inputStats = 6; + optional TableStatsProto resultStats = 7; + repeated ShuffleFileOutput shuffleFileOutputs = 8; +} + +message TaskCompletionReport { + required TaskAttemptIdProto id = 1; + optional StatSetProto stats = 2; + optional TableStatsProto inputStats = 3; + optional TableStatsProto resultStats = 4; + repeated ShuffleFileOutput shuffleFileOutputs = 5; +} + +message TaskFatalErrorReport { + required TaskAttemptIdProto id = 1; + optional string errorMessage = 2; + optional string errorTrace = 3; +} + +message FailureIntermediateProto { + required int64 pagePos = 1; + required int32 startRowNum = 2; + required int32 endRowNum = 3; +} + +message IntermediateEntryProto { + message PageProto { + required int64 pos = 1; + required int32 length = 2; + } + required ExecutionBlockIdProto ebId = 1; + required int32 taskId = 2; + required int32 attemptId = 3; + required int32 partId = 4; + required string host = 5; + required int64 volume = 6; + repeated PageProto pages = 7; + repeated FailureIntermediateProto failures = 8; +} + +message ExecutionBlockReport { + required ExecutionBlockIdProto ebId = 1; + required bool reportSuccess = 2; + optional string reportErrorMessage = 3; + required int32 succeededTasks = 4; + repeated IntermediateEntryProto intermediateEntries = 5; +} + +// deprecated +message TaskResponseProto { + required string id = 1; + required QueryState status = 2; +} + +message StatusReportProto { + required int64 timestamp = 1; + required string serverName = 2; + repeated TaskStatusProto status = 3; + repeated TaskAttemptIdProto pings = 4; +} + +message CommandRequestProto { + repeated Command command = 1; +} + +message CommandResponseProto { +} + +message Command { + required TaskAttemptIdProto id = 1; + required CommandType type = 2; +} + +enum CommandType { + PREPARE = 0; + LAUNCH = 1; + STOP = 2; + FINALIZE = 3; +} + +message ShuffleFileOutput { + required int32 partId = 1; + optional string fileName = 2; + optional int64 volume = 3; +} + +message SessionProto { + required string session_id = 1; + required string username = 2; + required string current_database = 3; + required int64 last_access_time = 4; + required KeyValueSetProto variables = 5; +} + +message NodeHeartbeatRequest { + required int32 workerId = 1; + optional NodeResourceProto totalResource = 2; + optional NodeResourceProto availableResource = 3; + optional int32 runningTasks = 4; + optional int32 runningQueryMasters = 5; + optional WorkerConnectionInfoProto connectionInfo = 6; + optional NodeStatusProto status = 7; +} + +message NodeHeartbeatResponse { + required ResponseCommand command = 1 [default = NORMAL]; + optional int32 heartBeatInterval = 2; + repeated QueryIdProto queryId = 3; +} + +//deplecated +message TajoHeartbeatRequest { + required WorkerConnectionInfoProto connectionInfo = 1; + optional QueryIdProto queryId = 2; + optional QueryState state = 3; + optional TableDescProto resultDesc = 4; + optional string statusMessage = 5; + optional float queryProgress = 6; +} + +//deplecated +message TajoHeartbeatResponse { + message ResponseCommand { + required string command = 1; + repeated string params = 2; + } + required BoolProto heartbeatResult = 1; + optional ResponseCommand responseCommand = 3; +} + +message WorkerConnectionsResponse { + repeated WorkerConnectionInfoProto worker = 1; +} + +message NodeResourceRequest { + optional string queue = 1; + required string userId = 2; + required ResourceType type = 3; + required int32 priority = 4; + required QueryIdProto queryId = 5; + required int32 numContainers = 6; + required NodeResourceProto capacity = 7; + required int32 runningTasks = 8; + repeated int32 candidateNodes = 9; +} + +message NodeResourceResponse { + required QueryIdProto queryId = 1; + repeated AllocationResourceProto resource = 2; +} + +message ExecutionBlockContextRequest { + required ExecutionBlockIdProto executionBlockId = 1; + required WorkerConnectionInfoProto worker = 2; +} + +message ExecutionBlockContextResponse { + required ExecutionBlockIdProto executionBlockId = 1; + optional string queryOutputPath = 2; + + required KeyValueSetProto queryContext = 3; + required string planJson = 4; + required ShuffleType shuffleType = 5; +} + +message StopExecutionBlockRequest { + required ExecutionBlockIdProto executionBlockId = 1; + optional ExecutionBlockListProto cleanupList = 2; +} + +message BatchAllocationRequest { + required ExecutionBlockIdProto executionBlockId = 1; + repeated TaskAllocationProto taskRequest = 2; +} + +message BatchAllocationResponse { + repeated TaskAllocationProto cancellationTask = 1; +} + +message QueryExecutionRequest { + required QueryIdProto queryId = 1; + required SessionProto session = 2; + required KeyValueSetProto queryContext = 3; + required StringProto exprInJson = 4; + optional StringProto logicalPlanJson = 5; + required AllocationResourceProto allocation = 6; +} + + +//Task history +message FetcherHistoryProto { + required int64 startTime = 1; + optional int64 finishTime = 2; + required FetcherState state = 3; + required int64 fileLength = 4; + required int32 messageReceivedCount = 5; +} + +message TaskHistoryProto { + required TaskAttemptIdProto taskAttemptId = 1; + required TaskAttemptState state = 2; + required float progress = 3; + required int64 startTime = 4; + required int64 finishTime = 5; + required TableStatsProto inputStats = 6; + optional TableStatsProto outputStats = 7; + optional string outputPath = 8; + optional string workingPath = 9; + optional int32 finishedFetchCount = 10; + optional int32 totalFetchCount = 11; + repeated FetcherHistoryProto fetcherHistories = 12; +} diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto index 17589d4613..e0ddac2324 100644 --- a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto +++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto @@ -22,39 +22,10 @@ option java_outer_classname = "TajoResourceTrackerProtocol"; option java_generic_services = true; option java_generate_equals_and_hash = true; -import "QueryCoordinatorProtocol.proto"; -import "tajo_protos.proto"; -import "TajoIdProtos.proto"; +import "ResourceProtos.proto"; package hadoop.yarn; -message NodeHeartbeatRequestProto { - required int32 workerId = 1; - optional NodeResourceProto totalResource = 2; - optional NodeResourceProto availableResource = 3; - optional int32 runningTasks = 4; - optional int32 runningQueryMasters = 5; - optional WorkerConnectionInfoProto connectionInfo = 6; - optional NodeStatusProto status = 7; -} - -message NodeHeartbeatResponseProto { - required ResponseCommand command = 1 [default = NORMAL]; - optional int32 heartBeatInterval = 2; - repeated QueryIdProto queryId = 3; -} - -enum ResponseCommand { - NORMAL = 1; //ping - MEMBERSHIP = 2; // request membership to worker node - ABORT_QUERY = 3; //query master failure - SHUTDOWN = 4; // black list -} - -//TODO add node health information -message NodeStatusProto { -} - service TajoResourceTrackerProtocolService { - rpc nodeHeartbeat(NodeHeartbeatRequestProto) returns (NodeHeartbeatResponseProto); + rpc nodeHeartbeat(NodeHeartbeatRequest) returns (NodeHeartbeatResponse); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 53c901da07..8667702b1b 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -22,357 +22,19 @@ 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"; -import "Plan.proto"; -import "QueryCoordinatorProtocol.proto"; +import "ResourceProtos.proto"; package hadoop.yarn; -message SessionProto { - required string session_id = 1; - required string username = 2; - required string current_database = 3; - required int64 last_access_time = 4; - required KeyValueSetProto variables = 5; -} - -message TaskStatusProto { - required TaskAttemptIdProto id = 1; - required string workerName = 2; - required float progress = 3; - required TaskAttemptState state = 4; - optional StatSetProto stats = 5; - optional TableStatsProto inputStats = 6; - optional TableStatsProto resultStats = 7; - repeated ShuffleFileOutput shuffleFileOutputs = 8; -} - -message TaskCompletionReport { - required TaskAttemptIdProto id = 1; - optional StatSetProto stats = 2; - optional TableStatsProto inputStats = 3; - optional TableStatsProto resultStats = 4; - repeated ShuffleFileOutput shuffleFileOutputs = 5; -} - -message TaskFatalErrorReport { - required TaskAttemptIdProto id = 1; - optional string errorMessage = 2; - optional string errorTrace = 3; -} - -message TaskRequestProto { - required string queryMasterHostAndPort = 1; - required TaskAttemptIdProto id = 2; - repeated FragmentProto fragments = 3; - required string outputTable = 4; - required bool clusteredOutput = 5; - required LogicalNodeTree plan = 6; - optional bool interQuery = 7 [default = false]; - repeated FetchProto fetches = 8; - optional KeyValueSetProto queryContext = 9; - optional DataChannelProto dataChannel = 10; - optional EnforcerProto enforcer = 11; -} - -message FetchProto { - required string host = 1; - required int32 port = 2; - required ShuffleType type = 3; - required ExecutionBlockIdProto executionBlockId = 4; - required int32 partitionId = 5; - required string name = 6; - optional string rangeParams = 7; - optional bool hasNext = 8 [default = false]; - - //repeated part - repeated int32 taskId = 9 [packed=true]; - repeated int32 attemptId = 10 [packed=true]; - - optional int64 offset = 11; - optional int64 length = 12; -} - -message FailureIntermediateProto { - required int64 pagePos = 1; - required int32 startRowNum = 2; - required int32 endRowNum = 3; -} - -message IntermediateEntryProto { - message PageProto { - required int64 pos = 1; - required int32 length = 2; - } - required ExecutionBlockIdProto ebId = 1; - required int32 taskId = 2; - required int32 attemptId = 3; - required int32 partId = 4; - required string host = 5; - required int64 volume = 6; - repeated PageProto pages = 7; - repeated FailureIntermediateProto failures = 8; -} - -message ExecutionBlockReport { - required ExecutionBlockIdProto ebId = 1; - required bool reportSuccess = 2; - optional string reportErrorMessage = 3; - required int32 succeededTasks = 4; - repeated IntermediateEntryProto intermediateEntries = 5; -} - -// deprecated -message TaskResponseProto { - required string id = 1; - required QueryState status = 2; -} - -message StatusReportProto { - required int64 timestamp = 1; - required string serverName = 2; - repeated TaskStatusProto status = 3; - repeated TaskAttemptIdProto pings = 4; -} - -message CommandRequestProto { - repeated Command command = 1; -} - -message CommandResponseProto { -} - -message Command { - required TaskAttemptIdProto id = 1; - required CommandType type = 2; -} - -enum CommandType { - PREPARE = 0; - LAUNCH = 1; - STOP = 2; - FINALIZE = 3; -} - -message ShuffleFileOutput { - required int32 partId = 1; - optional string fileName = 2; - optional int64 volume = 3; -} - -message QueryExecutionRequestProto { - required QueryIdProto queryId = 1; - required SessionProto session = 2; - required KeyValueSetProto queryContext = 3; - required StringProto exprInJson = 4; - optional StringProto logicalPlanJson = 5; - required AllocationResourceProto allocation = 6; -} - -message DataChannelProto { - required ExecutionBlockIdProto srcId = 1; - required ExecutionBlockIdProto targetId = 2; - - required TransmitType transmitType = 3 [default = PULL_TRANSMIT]; - required ShuffleType shuffleType = 4; - - optional SchemaProto schema = 5; - - repeated ColumnProto shuffleKeys = 7; - optional int32 numOutputs = 9 [default = 1]; - - optional string storeType = 10; -} - - -message ExecutionBlockContextProto { - required ExecutionBlockIdProto executionBlockId = 1; - optional string queryOutputPath = 2; - - required KeyValueSetProto queryContext = 3; - required string planJson = 4; - required ShuffleType shuffleType = 5; -} - -message ExecutionBlockContextRequestProto { - required ExecutionBlockIdProto executionBlockId = 1; - required WorkerConnectionInfoProto worker = 2; -} - -message StopExecutionBlockRequestProto { - required ExecutionBlockIdProto executionBlockId = 1; - optional ExecutionBlockListProto cleanupList = 2; -} - -message ExecutionBlockListProto { - repeated ExecutionBlockIdProto executionBlockId = 1; -} - -message TaskAllocationRequestProto { - required TaskRequestProto taskRequest = 1; - required NodeResourceProto resource = 2; -} - -message BatchAllocationRequestProto { - required ExecutionBlockIdProto executionBlockId = 1; - repeated TaskAllocationRequestProto taskRequest = 2; -} - -message BatchAllocationResponseProto { - repeated TaskAllocationRequestProto cancellationTask = 1; -} - service TajoWorkerProtocolService { rpc ping (TaskAttemptIdProto) returns (BoolProto); // from QueryMaster(Worker) - rpc allocateTasks(BatchAllocationRequestProto) returns (BatchAllocationResponseProto); - rpc stopExecutionBlock(StopExecutionBlockRequestProto) returns (BoolProto); + rpc allocateTasks(BatchAllocationRequest) returns (BatchAllocationResponse); + rpc stopExecutionBlock(StopExecutionBlockRequest) returns (BoolProto); rpc killTaskAttempt(TaskAttemptIdProto) returns (BoolProto); rpc stopQuery(QueryIdProto) returns (BoolProto); -} - -message EnforceProperty { - enum EnforceType { - SORTED_INPUT = 0; - OUTPUT_DISTINCT = 1; - GROUP_BY = 2; - JOIN = 3; - SORT = 4; - BROADCAST = 5; - COLUMN_PARTITION = 6; - DISTINCT_GROUP_BY = 7; - } - - // Identifies which field is filled in. - required EnforceType type = 1; - - // One of the following will be filled in. - optional SortedInputEnforce sortedInput = 2; - optional OutputDistinctEnforce outputDistinct = 3; - optional GroupbyEnforce groupby = 4; - optional JoinEnforce join = 5; - optional SortEnforce sort = 6; - optional BroadcastEnforce broadcast = 7; - optional ColumnPartitionEnforcer columnPartition = 8; - optional DistinctGroupbyEnforcer distinct = 9; -} - -message SortedInputEnforce { - required string tableName = 1; - repeated SortSpecProto sortSpecs = 2; -} - -message OutputDistinctEnforce { -} - -message JoinEnforce { - enum JoinAlgorithm { - NESTED_LOOP_JOIN = 0; - BLOCK_NESTED_LOOP_JOIN = 1; - IN_MEMORY_HASH_JOIN = 2; - HYBRID_HASH_JOIN = 3; - MERGE_JOIN = 4; - } - - required int32 nodeId = 1; - required JoinAlgorithm algorithm = 2; -} - -message GroupbyEnforce { - enum GroupbyAlgorithm { - HASH_AGGREGATION = 0; - SORT_AGGREGATION = 1; - } - - required int32 nodeId = 1; - required GroupbyAlgorithm algorithm = 2; - repeated SortSpecProto sortSpecs = 3; -} - -message SortEnforce { - enum SortAlgorithm { - IN_MEMORY_SORT = 0; - MERGE_SORT = 1; - } - - required int32 nodeId = 1; - required SortAlgorithm algorithm = 2; -} - -message BroadcastEnforce { - required string tableName = 1; -} - -message ColumnPartitionEnforcer { - enum ColumnPartitionAlgorithm { - HASH_PARTITION = 0; - SORT_PARTITION = 1; - } - - required int32 nodeId = 1; - required ColumnPartitionAlgorithm algorithm = 2; -} - -message DistinctGroupbyEnforcer { - enum DistinctAggregationAlgorithm { - HASH_AGGREGATION = 0; - SORT_AGGREGATION = 1; - } - - enum MultipleAggregationStage { - FIRST_STAGE = 0; - SECOND_STAGE = 1; - THRID_STAGE = 3; - } - - message SortSpecArray { - required int32 nodeId = 1; - repeated SortSpecProto sortSpecs = 2; - } - required int32 nodeId = 1; - required DistinctAggregationAlgorithm algorithm = 2; - repeated SortSpecArray sortSpecArrays = 3; - required bool isMultipleAggregation = 4 [default = false]; - optional MultipleAggregationStage multipleAggregationStage = 5; -} - -message EnforcerProto { - repeated EnforceProperty properties = 1; -} - -message FetcherHistoryProto { - required int64 startTime = 1; - optional int64 finishTime = 2; - required FetcherState state = 3; - required int64 fileLength = 4; - required int32 messageReceivedCount = 5; -} - -message TaskHistoryProto { - required TaskAttemptIdProto taskAttemptId = 1; - required TaskAttemptState state = 2; - required float progress = 3; - required int64 startTime = 4; - required int64 finishTime = 5; - required TableStatsProto inputStats = 6; - optional TableStatsProto outputStats = 7; - optional string outputPath = 8; - optional string workingPath = 9; - optional int32 finishedFetchCount = 10; - optional int32 totalFetchCount = 11; - repeated FetcherHistoryProto fetcherHistories = 12; -} - -message TaskRunnerHistoryProto { - required ExecutionBlockIdProto executionBlockId = 1; - required string state = 2; - required string containerId = 3; - optional int64 startTime = 4; - optional int64 finishTime = 5; - repeated TaskHistoryProto taskHistories = 6; } \ No newline at end of file diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java index ace3d0d901..d1983266f2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java @@ -50,7 +50,7 @@ import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java index dc4dd042b8..3c9177ebdc 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java @@ -50,7 +50,7 @@ import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestFullOuterHashJoinExec { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java index 8fd61d0de0..6b32db0c6c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java @@ -52,7 +52,7 @@ import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestFullOuterMergeJoinExec { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java index b9ee06a0f8..f7bc1dde21 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java @@ -51,7 +51,7 @@ import java.io.IOException; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestHashJoinExec { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java index c93a1b40b6..87bcd20539 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java @@ -50,7 +50,7 @@ import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java index c4e7752ea6..d4189bb505 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java @@ -52,7 +52,7 @@ import java.io.IOException; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java index dff0cbec7d..ca4b6b79d2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java @@ -69,8 +69,8 @@ import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm; -import static org.apache.tajo.ipc.TajoWorkerProtocol.SortEnforce.SortAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.ColumnPartitionEnforcer.ColumnPartitionAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.SortEnforce.SortAlgorithm; import static org.junit.Assert.*; public class TestPhysicalPlanner { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java index d86b2294bb..51f0e764a0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java @@ -50,7 +50,7 @@ import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; -import static org.apache.tajo.ipc.TajoWorkerProtocol.JoinEnforce.JoinAlgorithm; +import static org.apache.tajo.plan.serder.PlanProto.JoinEnforce.JoinAlgorithm; import static org.junit.Assert.*; public class TestRightOuterMergeJoinExec { diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java index 9910d79f27..3c378dd969 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java @@ -20,29 +20,28 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import io.netty.handler.codec.http.QueryStringDecoder; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.LocalTajoTestingUtility; import org.apache.tajo.QueryId; +import org.apache.tajo.ResourceProtos.FetchProto; import org.apache.tajo.TestTajoIds; -import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.querymaster.Repartitioner; import org.apache.tajo.querymaster.Task; import org.apache.tajo.querymaster.Task.IntermediateEntry; -import org.apache.tajo.querymaster.Repartitioner; import org.apache.tajo.util.Pair; import org.apache.tajo.util.TUtil; import org.apache.tajo.worker.FetchImpl; import org.junit.Test; -import io.netty.handler.codec.http.QueryStringDecoder; - import java.net.URI; import java.util.*; import static junit.framework.Assert.assertEquals; -import static org.apache.tajo.querymaster.Repartitioner.FetchGroupMeta; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType.HASH_SHUFFLE; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType.SCATTERED_HASH_SHUFFLE; +import static org.apache.tajo.querymaster.Repartitioner.FetchGroupMeta; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; @@ -76,7 +75,7 @@ public void testCreateHashFetchURL() throws Exception { fetch.setName(sid.toString()); - TajoWorkerProtocol.FetchProto proto = fetch.getProto(); + FetchProto proto = fetch.getProto(); fetch = new FetchImpl(proto); assertEquals(proto, fetch.getProto()); diff --git a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java index 6dcbd95c28..af5b1cf93c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java @@ -44,7 +44,7 @@ import java.util.Map; import java.util.concurrent.*; -import static org.apache.tajo.ipc.QueryCoordinatorProtocol.*; +import static org.apache.tajo.ResourceProtos.*; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -179,11 +179,11 @@ public void testReserveResource() throws InterruptedException, ExecutionExceptio assertEquals(totalResource, scheduler.getClusterResource()); QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); - CallFuture callBack = new CallFuture(); + CallFuture callBack = new CallFuture(); rmContext.getDispatcher().getEventHandler().handle(new ResourceReserveSchedulerEvent( createResourceRequest(queryId, requestNum, new ArrayList()), callBack)); - NodeResourceResponseProto responseProto = callBack.get(); + NodeResourceResponse responseProto = callBack.get(); assertEquals(queryId, new QueryId(responseProto.getQueryId())); assertEquals(requestNum, responseProto.getResourceCount()); @@ -209,12 +209,12 @@ public void testReserveResourceWithWorkerPriority() throws InterruptedException, assertTrue(NodeResources.fitsIn(expectResource, workerEntry.getValue().getAvailableResource())); QueryId queryId = QueryIdFactory.newQueryId(System.nanoTime(), 0); - NodeResourceRequestProto requestProto = createResourceRequest(queryId, requestNum, targetWorkers); - CallFuture callBack = new CallFuture(); + NodeResourceRequest requestProto = createResourceRequest(queryId, requestNum, targetWorkers); + CallFuture callBack = new CallFuture(); rmContext.getDispatcher().getEventHandler().handle(new ResourceReserveSchedulerEvent( requestProto, callBack)); - NodeResourceResponseProto responseProto = callBack.get(); + NodeResourceResponse responseProto = callBack.get(); assertEquals(queryId, new QueryId(responseProto.getQueryId())); assertEquals(requestNum, responseProto.getResourceCount()); @@ -223,10 +223,10 @@ public void testReserveResourceWithWorkerPriority() throws InterruptedException, } } - private NodeResourceRequestProto + private NodeResourceRequest createResourceRequest(QueryId queryId, int containerNum, List candidateWorkers) { - NodeResourceRequestProto.Builder request = - NodeResourceRequestProto.newBuilder(); + NodeResourceRequest.Builder request = + NodeResourceRequest.newBuilder(); request.setCapacity(scheduler.getMinimumResourceCapability().getProto()) .setNumContainers(containerNum) .setPriority(1) diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index 9b4c06997d..3e8e230278 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.tajo.*; +import org.apache.tajo.ResourceProtos.ExecutionBlockContextResponse; import org.apache.tajo.algebra.Expr; import org.apache.tajo.benchmark.TPCH; import org.apache.tajo.catalog.CatalogService; @@ -34,7 +35,6 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequestImpl; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.QueryEvent; import org.apache.tajo.master.event.QueryEventType; @@ -243,8 +243,8 @@ public void testKillTask() throws Throwable { taskRequest.setInterQuery(); - TajoWorkerProtocol.ExecutionBlockContextProto.Builder requestProtoBuilder = - TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); + ExecutionBlockContextResponse.Builder requestProtoBuilder = + ExecutionBlockContextResponse.newBuilder(); requestProtoBuilder.setExecutionBlockId(eid.getProto()) .setPlanJson("test") .setQueryContext(new QueryContext(conf).getProto()) diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java index 8abcb8f95c..7d7fb1a1a0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java @@ -18,15 +18,15 @@ package org.apache.tajo.worker; +import org.apache.tajo.ResourceProtos.ExecutionBlockContextResponse; import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.ipc.TajoWorkerProtocol; import java.io.IOException; public class MockExecutionBlock extends ExecutionBlockContext { public MockExecutionBlock(TajoWorker.WorkerContext workerContext, - TajoWorkerProtocol.ExecutionBlockContextProto request) throws IOException { + ExecutionBlockContextResponse request) throws IOException { super(workerContext, request, null); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java index 75c89bbb71..8c8427d37b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java @@ -21,8 +21,9 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.QueryIdFactory; +import org.apache.tajo.ResourceProtos.TaskAllocationProto; +import org.apache.tajo.ResourceProtos.TaskRequestProto; import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.plan.serder.PlanProto; import org.apache.tajo.resource.NodeResource; import org.apache.tajo.resource.NodeResources; @@ -49,7 +50,7 @@ public void handle(NodeResourceEvent event) { } @Override - protected void startTask(TajoWorkerProtocol.TaskRequestProto request, NodeResource resource) { + protected void startTask(TaskRequestProto request, NodeResource resource) { if(enableTaskHandlerEvent) { super.startTask(request, resource); } @@ -62,16 +63,15 @@ public void setTaskHandlerEvent(boolean flag) { enableTaskHandlerEvent = flag; } - protected static Queue createTaskRequests( + protected static Queue createTaskRequests( ExecutionBlockId ebId, int memory, int size) { - Queue - requestProtoList = new LinkedBlockingQueue(); + Queue + requestProtoList = new LinkedBlockingQueue(); for (int i = 0; i < size; i++) { TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(ebId, i), 0); - TajoWorkerProtocol.TaskRequestProto.Builder builder = - TajoWorkerProtocol.TaskRequestProto.newBuilder(); + TaskRequestProto.Builder builder = TaskRequestProto.newBuilder(); builder.setQueryMasterHostAndPort("localhost:0"); builder.setId(taskAttemptId.getProto()); builder.setOutputTable(""); @@ -79,7 +79,7 @@ protected static Queue createTask builder.setClusteredOutput(false); - requestProtoList.add(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + requestProtoList.add(TaskAllocationProto.newBuilder() .setResource(NodeResources.createResource(memory).getProto()) .setTaskRequest(builder.build()).build()); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java index 80c1604202..634398f914 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java @@ -21,8 +21,7 @@ import com.google.common.collect.Maps; import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; -import org.apache.tajo.ipc.QueryCoordinatorProtocol; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.ipc.TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService; import org.apache.tajo.resource.NodeResource; import org.apache.tajo.resource.NodeResources; @@ -30,7 +29,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; -import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; +import static org.apache.tajo.ResourceProtos.*; public class MockNodeStatusUpdater extends NodeStatusUpdater { @@ -57,7 +56,7 @@ protected MockResourceTracker getResourceTracker() { } class MockResourceTracker implements TajoResourceTrackerProtocolService.Interface { - private NodeHeartbeatRequestProto lastRequest; + private NodeHeartbeatRequest lastRequest; protected Map getTotalResource() { return membership; @@ -67,15 +66,15 @@ protected Map getAvailableResource() { return membership; } - protected NodeHeartbeatRequestProto getLastRequest() { + protected NodeHeartbeatRequest getLastRequest() { return lastRequest; } @Override - public void nodeHeartbeat(RpcController controller, NodeHeartbeatRequestProto request, - RpcCallback done) { + public void nodeHeartbeat(RpcController controller, NodeHeartbeatRequest request, + RpcCallback done) { - NodeHeartbeatResponseProto.Builder response = NodeHeartbeatResponseProto.newBuilder(); + NodeHeartbeatResponse.Builder response = NodeHeartbeatResponse.newBuilder(); if (membership.containsKey(request.getWorkerId())) { if (request.hasAvailableResource()) { NodeResource resource = resources.get(request.getWorkerId()); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java index 90e752093a..23405510f8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -18,10 +18,11 @@ package org.apache.tajo.worker; +import org.apache.tajo.ResourceProtos.TaskRequestProto; +import org.apache.tajo.ResourceProtos.TaskStatusProto; import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.worker.event.TaskStartEvent; import java.io.IOException; @@ -44,7 +45,7 @@ public void handle(TaskStartEvent event) { } @Override - protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + protected Task createTask(final ExecutionBlockContext context, TaskRequestProto taskRequest) { final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); //ignore status changed log @@ -126,8 +127,8 @@ public ExecutionBlockContext getExecutionBlockContext() { } @Override - public TajoWorkerProtocol.TaskStatusProto getReport() { - TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + public TaskStatusProto getReport() { + TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); builder.setWorkerName("localhost:0"); builder.setId(taskAttemptContext.getTaskId().getProto()) .setProgress(taskAttemptContext.getProgress()) diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java index cad8063df2..76ce9f7591 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java @@ -20,6 +20,8 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.ResourceProtos.ExecutionBlockContextResponse; +import org.apache.tajo.ResourceProtos.ExecutionBlockListProto; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; @@ -41,8 +43,7 @@ public MockTaskManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.Work @Override protected ExecutionBlockContext createExecutionBlock(ExecutionBlockId executionBlockId, String queryMaster) { try { - TajoWorkerProtocol.ExecutionBlockContextProto.Builder builder = - TajoWorkerProtocol.ExecutionBlockContextProto.newBuilder(); + ExecutionBlockContextResponse.Builder builder = ExecutionBlockContextResponse.newBuilder(); builder.setExecutionBlockId(executionBlockId.getProto()) .setPlanJson("test") .setQueryContext(new QueryContext(new TajoConf()).getProto()) @@ -56,7 +57,7 @@ protected ExecutionBlockContext createExecutionBlock(ExecutionBlockId executionB @Override protected void stopExecutionBlock(ExecutionBlockContext context, - TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + ExecutionBlockListProto cleanupList) { //skip for testing } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index ac25cdbaee..a9bd849ed7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -40,7 +40,7 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.apache.tajo.ResourceProtos.*; import static org.junit.Assert.*; public class TestNodeResourceManager { @@ -134,8 +134,8 @@ public void testNodeResourceAllocateEvent() throws Exception { int requestSize = 4; resourceManager.setTaskHandlerEvent(false); //skip task execution - CallFuture callFuture = new CallFuture(); - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + CallFuture callFuture = new CallFuture(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); requestProto.setExecutionBlockId(ebId.getProto()); @@ -144,7 +144,7 @@ public void testNodeResourceAllocateEvent() throws Exception { dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); - BatchAllocationResponseProto responseProto = callFuture.get(); + BatchAllocationResponse responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); // allocated all assertEquals(0, responseProto.getCancellationTaskCount()); @@ -157,8 +157,8 @@ public void testNodeResourceCancellation() throws Exception { int overSize = 10; resourceManager.setTaskHandlerEvent(false); //skip task execution - CallFuture callFuture = new CallFuture(); - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + CallFuture callFuture = new CallFuture(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); requestProto.setExecutionBlockId(ebId.getProto()); @@ -167,7 +167,7 @@ public void testNodeResourceCancellation() throws Exception { MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize + overSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); - BatchAllocationResponseProto responseProto = callFuture.get(); + BatchAllocationResponse responseProto = callFuture.get(); assertEquals(overSize, responseProto.getCancellationTaskCount()); } @@ -177,8 +177,8 @@ public void testNodeResourceDeallocateEvent() throws Exception { int requestSize = 4; resourceManager.setTaskHandlerEvent(false); //skip task execution - CallFuture callFuture = new CallFuture(); - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + CallFuture callFuture = new CallFuture(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); requestProto.setExecutionBlockId(ebId.getProto()); @@ -187,12 +187,12 @@ public void testNodeResourceDeallocateEvent() throws Exception { dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); - BatchAllocationResponseProto responseProto = callFuture.get(); + BatchAllocationResponse responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); assertEquals(0, responseProto.getCancellationTaskCount()); //deallocate - for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { + for(TaskAllocationProto allocationRequestProto : requestProto.getTaskRequestList()) { // direct invoke handler for testing resourceManager.handle(new NodeResourceDeallocateEvent( allocationRequestProto.getResource(), NodeResourceEvent.ResourceType.TASK)); @@ -211,15 +211,15 @@ public void testParallelRequest() throws Exception { final AtomicInteger totalCanceled = new AtomicInteger(); final ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); - final Queue + final Queue totalTasks = MockNodeResourceManager.createTaskRequests(ebId, taskMemory, taskSize); - TaskAllocationRequestProto task = totalTasks.poll(); - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + TaskAllocationProto task = totalTasks.poll(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); requestProto.addTaskRequest(task); requestProto.setExecutionBlockId(ebId.getProto()); - CallFuture callFuture = new CallFuture(); + CallFuture callFuture = new CallFuture(); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); assertTrue(callFuture.get().getCancellationTaskCount() == 0); totalComplete.incrementAndGet(); @@ -236,18 +236,18 @@ public void testParallelRequest() throws Exception { public void run() { int complete = 0; while (true) { - TaskAllocationRequestProto task = totalTasks.poll(); + TaskAllocationProto task = totalTasks.poll(); if (task == null) break; - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); requestProto.addTaskRequest(task); requestProto.setExecutionBlockId(ebId.getProto()); - CallFuture callFuture = new CallFuture(); + CallFuture callFuture = new CallFuture(); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); try { - BatchAllocationResponseProto proto = callFuture.get(); + BatchAllocationResponse proto = callFuture.get(); if (proto.getCancellationTaskCount() > 0) { totalTasks.addAll(proto.getCancellationTaskList()); totalCanceled.addAndGet(proto.getCancellationTaskCount()); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 4729c053f3..a53bb6069d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -33,6 +33,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; +import static org.apache.tajo.ResourceProtos.*; import static org.junit.Assert.*; public class TestNodeStatusUpdater { @@ -143,7 +144,7 @@ public void testPing() throws Exception { MockNodeStatusUpdater.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); barrier.await(); - TajoResourceTrackerProtocol.NodeHeartbeatRequestProto lastRequest = resourceTracker.getLastRequest(); + NodeHeartbeatRequest lastRequest = resourceTracker.getLastRequest(); assertTrue(lastRequest.hasWorkerId()); assertTrue(lastRequest.hasAvailableResource()); assertTrue(lastRequest.hasRunningTasks()); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index ca19f6f5f9..da596093c2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -24,7 +24,6 @@ import org.apache.tajo.*; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.util.CommonTestingUtil; @@ -40,8 +39,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; -import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; +import static org.apache.tajo.ResourceProtos.*; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -142,8 +140,8 @@ public void testTaskRequest() throws Exception { QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); - CallFuture callFuture = new CallFuture(); - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + CallFuture callFuture = new CallFuture(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); @@ -170,8 +168,8 @@ public void testTaskException() throws Exception { QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); - CallFuture callFuture = new CallFuture(); - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + CallFuture callFuture = new CallFuture(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); @@ -207,7 +205,7 @@ protected void stopTask(TaskAttemptId taskId) { } @Override - protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + protected Task createTask(final ExecutionBlockContext context, TaskRequestProto taskRequest) { final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null); @@ -288,8 +286,8 @@ public ExecutionBlockContext getExecutionBlockContext() { } @Override - public TajoWorkerProtocol.TaskStatusProto getReport() { - TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + public TaskStatusProto getReport() { + TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); builder.setWorkerName("localhost:0"); builder.setId(taskAttemptContext.getTaskId().getProto()) .setProgress(taskAttemptContext.getProgress()) diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index 73474cfcee..8d468c19a3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -41,7 +41,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.apache.tajo.ResourceProtos.*; import static org.junit.Assert.*; public class TestTaskManager { @@ -146,8 +146,8 @@ public void testExecutionBlockStart() throws Exception { QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); - CallFuture callFuture = new CallFuture(); - BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + CallFuture callFuture = new CallFuture(); + BatchAllocationRequest.Builder requestProto = BatchAllocationRequest.newBuilder(); requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); @@ -165,7 +165,7 @@ public void testExecutionBlockStop() throws Exception { QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); - TaskAllocationRequestProto requestProto = + TaskAllocationProto requestProto = MockNodeResourceManager.createTaskRequests(ebId, taskMemory, 1).poll(); taskDispatcher.getEventHandler().handle(new TaskStartEvent(requestProto.getTaskRequest(), diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto index 40b789118c..9ba1c83785 100644 --- a/tajo-plan/src/main/proto/Plan.proto +++ b/tajo-plan/src/main/proto/Plan.proto @@ -24,6 +24,7 @@ option java_generate_equals_and_hash = true; import "PrimitiveProtos.proto"; import "CatalogProtos.proto"; import "DataTypes.proto"; +import "TajoIdProtos.proto"; enum NodeType { SET_SESSION = 0; @@ -495,4 +496,128 @@ enum TransmitType { PUSH_TRANSMIT = 0; PULL_TRANSMIT = 1; FILE_WRITE = 2; +} + +message DataChannelProto { + required ExecutionBlockIdProto srcId = 1; + required ExecutionBlockIdProto targetId = 2; + + required TransmitType transmitType = 3 [default = PULL_TRANSMIT]; + required ShuffleType shuffleType = 4; + + optional SchemaProto schema = 5; + + repeated ColumnProto shuffleKeys = 7; + optional int32 numOutputs = 9 [default = 1]; + + optional string storeType = 10; +} + +message EnforcerProto { + repeated EnforceProperty properties = 1; +} + +message EnforceProperty { + enum EnforceType { + SORTED_INPUT = 0; + OUTPUT_DISTINCT = 1; + GROUP_BY = 2; + JOIN = 3; + SORT = 4; + BROADCAST = 5; + COLUMN_PARTITION = 6; + DISTINCT_GROUP_BY = 7; + } + + // Identifies which field is filled in. + required EnforceType type = 1; + + // One of the following will be filled in. + optional SortedInputEnforce sortedInput = 2; + optional OutputDistinctEnforce outputDistinct = 3; + optional GroupbyEnforce groupby = 4; + optional JoinEnforce join = 5; + optional SortEnforce sort = 6; + optional BroadcastEnforce broadcast = 7; + optional ColumnPartitionEnforcer columnPartition = 8; + optional DistinctGroupbyEnforcer distinct = 9; +} + +message SortedInputEnforce { + required string tableName = 1; + repeated SortSpecProto sortSpecs = 2; +} + +message OutputDistinctEnforce { +} + +message JoinEnforce { + enum JoinAlgorithm { + NESTED_LOOP_JOIN = 0; + BLOCK_NESTED_LOOP_JOIN = 1; + IN_MEMORY_HASH_JOIN = 2; + HYBRID_HASH_JOIN = 3; + MERGE_JOIN = 4; + } + + required int32 nodeId = 1; + required JoinAlgorithm algorithm = 2; +} + +message GroupbyEnforce { + enum GroupbyAlgorithm { + HASH_AGGREGATION = 0; + SORT_AGGREGATION = 1; + } + + required int32 nodeId = 1; + required GroupbyAlgorithm algorithm = 2; + repeated SortSpecProto sortSpecs = 3; +} + +message SortEnforce { + enum SortAlgorithm { + IN_MEMORY_SORT = 0; + MERGE_SORT = 1; + } + + required int32 nodeId = 1; + required SortAlgorithm algorithm = 2; +} + +message BroadcastEnforce { + required string tableName = 1; +} + +message ColumnPartitionEnforcer { + enum ColumnPartitionAlgorithm { + HASH_PARTITION = 0; + SORT_PARTITION = 1; + } + + required int32 nodeId = 1; + required ColumnPartitionAlgorithm algorithm = 2; +} + +message DistinctGroupbyEnforcer { + enum DistinctAggregationAlgorithm { + HASH_AGGREGATION = 0; + SORT_AGGREGATION = 1; + } + + enum MultipleAggregationStage { + FIRST_STAGE = 0; + SECOND_STAGE = 1; + THRID_STAGE = 3; + } + + message SortSpecArray { + required int32 nodeId = 1; + repeated SortSpecProto sortSpecs = 2; + } + required int32 nodeId = 1; + required DistinctAggregationAlgorithm algorithm = 2; + repeated SortSpecArray sortSpecArrays = 3; + required bool isMultipleAggregation = 4 [default = false]; + optional MultipleAggregationStage multipleAggregationStage = 5; } \ No newline at end of file From dc69501e2e9d26efe9da269dd54085bfb5a1ccd1 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 16 Jul 2015 17:28:47 +0900 Subject: [PATCH 67/80] add more comments --- .../main/java/org/apache/tajo/util/NumberUtil.java | 4 ++++ .../tajo/master/QueryCoordinatorService.java | 6 ++++++ .../org/apache/tajo/master/QueryInProgress.java | 3 +++ .../tajo/master/scheduler/QuerySchedulingInfo.java | 14 ++++++++++++-- .../tajo/master/scheduler/SimpleScheduler.java | 4 ++++ 5 files changed, 29 insertions(+), 2 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java index 0d70cc2034..32e086caf9 100644 --- a/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java +++ b/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java @@ -1050,4 +1050,8 @@ public static Number numberValue(Class numberClazz, String value) { return returnNumber; } + + public static int compare(long x, long y) { + return (x < y) ? -1 : ((x == y) ? 0 : 1); + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java index 9e508e4765..6457e6106b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java @@ -114,6 +114,9 @@ public void heartbeat( done.run(builder.build()); } + /** + * Reserve a node resources to TajoMaster + */ @Override public void reserveNodeResources(RpcController controller, NodeResourceRequest request, RpcCallback done) { @@ -121,6 +124,9 @@ public void reserveNodeResources(RpcController controller, NodeResourceRequest r dispatcher.getEventHandler().handle(new ResourceReserveSchedulerEvent(request, done)); } + /** + * Get all worker connection information + */ @Override public void getAllWorkers(RpcController controller, PrimitiveProtos.NullProto request, RpcCallback done) { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java index 6265b01ca7..0762589c27 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java @@ -184,6 +184,9 @@ private void connectQueryMaster(WorkerConnectionInfo connectionInfo) queryMasterRpcClient = queryMasterRpc.getStub(); } + /** + * Launch the allocated query to QueryMaster + */ public boolean submitToQueryMaster() { if(querySubmitted) { return false; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java index 16d424c9df..a1fe743499 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java @@ -20,12 +20,22 @@ import com.google.common.base.Objects; import org.apache.tajo.QueryId; +import org.apache.tajo.util.NumberUtil; + +/** + * A QuerySchedulingInfo represents an scheduling information. + * It provides a common interface for queue and priority + */ public class QuerySchedulingInfo implements Comparable { + /** Name of queue */ private String queue; + /** Query owner */ private String user; private QueryId queryId; + /** Query priority for queries in same queue */ private int priority; + /** Start time for query in same queue */ private long startTime; public QuerySchedulingInfo(String queue, String user, QueryId queryId, int priority, long startTime) { @@ -63,9 +73,9 @@ public String getQueue() { @Override public int compareTo(QuerySchedulingInfo o) { - int ret = Integer.compare(priority, o.priority); + int ret = NumberUtil.compare(priority, o.priority); if(ret == 0) { - ret = Long.compare(startTime, o.startTime); + ret = NumberUtil.compare(startTime, o.startTime); } return ret; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 0163d12745..2201873d8c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -46,6 +46,10 @@ import static org.apache.tajo.ResourceProtos.*; +/** + * SimpleScheduler can execute query and stages simultaneously. + * Each query and the stage competes to get the resource + */ public class SimpleScheduler extends AbstractQueryScheduler { private static final Log LOG = LogFactory.getLog(SimpleScheduler.class); From 060d1bf2354762041456888836abc3183dc64c21 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 16 Jul 2015 20:14:49 +0900 Subject: [PATCH 68/80] add missing stop --- .../org/apache/tajo/util/metrics/TestSystemMetrics.java | 1 + .../org/apache/tajo/worker/TestNodeResourceManager.java | 6 ++++++ .../org/apache/tajo/worker/TestNodeStatusUpdater.java | 9 +++++++-- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/util/metrics/TestSystemMetrics.java b/tajo-core/src/test/java/org/apache/tajo/util/metrics/TestSystemMetrics.java index 51622b55be..0ce26464dd 100644 --- a/tajo-core/src/test/java/org/apache/tajo/util/metrics/TestSystemMetrics.java +++ b/tajo-core/src/test/java/org/apache/tajo/util/metrics/TestSystemMetrics.java @@ -124,6 +124,7 @@ public void testMetricsReporter() throws Exception { } assertEquals(2, lines.size()); + tajoSystemMetrics.stop(); } @After diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index a9bd849ed7..b7c3967d7f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -118,6 +118,12 @@ protected void serviceInit(Configuration conf) throws Exception { addIfService(statusUpdater); super.serviceInit(conf); } + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } }; service.init(conf); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index a53bb6069d..db8aa737e3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -22,7 +22,6 @@ import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeStatusEvent; @@ -33,7 +32,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; -import static org.apache.tajo.ResourceProtos.*; +import static org.apache.tajo.ResourceProtos.NodeHeartbeatRequest; import static org.junit.Assert.*; public class TestNodeStatusUpdater { @@ -105,6 +104,12 @@ protected void serviceInit(Configuration conf) throws Exception { addIfService(statusUpdater); super.serviceInit(conf); } + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } }; service.init(conf); From c272ec79b6c779a098b66e311374f65567c1bbbe Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 17 Jul 2015 12:21:21 +0900 Subject: [PATCH 69/80] fix broken tajo-web --- .../main/resources/webapps/worker/task.jsp | 8 ++-- .../resources/webapps/worker/taskdetail.jsp | 11 ++++-- .../resources/webapps/worker/taskhistory.jsp | 6 +-- .../tajo/querymaster/TestQueryState.java | 38 +++++++++++-------- 4 files changed, 36 insertions(+), 27 deletions(-) diff --git a/tajo-core/src/main/resources/webapps/worker/task.jsp b/tajo-core/src/main/resources/webapps/worker/task.jsp index fcab84e62e..f2f903b2cb 100644 --- a/tajo-core/src/main/resources/webapps/worker/task.jsp +++ b/tajo-core/src/main/resources/webapps/worker/task.jsp @@ -21,15 +21,16 @@ <%@ page import="org.apache.tajo.ExecutionBlockId" %> <%@ page import="org.apache.tajo.QueryId" %> +<%@ page import="org.apache.tajo.ResourceProtos.ShuffleFileOutput" %> <%@ page import="org.apache.tajo.TaskId" %> <%@ page import="org.apache.tajo.catalog.proto.CatalogProtos" %> <%@ page import="org.apache.tajo.catalog.statistics.TableStats" %> -<%@ page import="org.apache.tajo.ipc.TajoWorkerProtocol" %> <%@ page import="org.apache.tajo.querymaster.Query" %> <%@ page import="org.apache.tajo.querymaster.QueryMasterTask" %> -<%@ page import="org.apache.tajo.querymaster.Task" %> <%@ page import="org.apache.tajo.querymaster.Stage" %> +<%@ page import="org.apache.tajo.querymaster.Task" %> <%@ page import="org.apache.tajo.storage.DataLocation" %> +<%@ page import="org.apache.tajo.storage.fragment.Fragment" %> <%@ page import="org.apache.tajo.storage.fragment.FragmentConvertor" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.TajoIdUtils" %> @@ -40,7 +41,6 @@ <%@ page import="java.text.SimpleDateFormat" %> <%@ page import="java.util.Map" %> <%@ page import="java.util.Set" %> -<%@ page import="org.apache.tajo.storage.fragment.Fragment" %> <% String paramQueryId = request.getParameter("queryId"); @@ -131,7 +131,7 @@ String shuffleKey = "-"; String shuffleFileName = "-"; if(numShuffles > 0) { - TajoWorkerProtocol.ShuffleFileOutput shuffleFileOutputs = task.getShuffleFileOutputs().get(0); + ShuffleFileOutput shuffleFileOutputs = task.getShuffleFileOutputs().get(0); shuffleKey = "" + shuffleFileOutputs.getPartId(); shuffleFileName = shuffleFileOutputs.getFileName(); } diff --git a/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp index ad147f6a3f..88b7c2430e 100644 --- a/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp +++ b/tajo-core/src/main/resources/webapps/worker/taskdetail.jsp @@ -20,12 +20,15 @@ <%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%> <%@ page import="org.apache.commons.lang.StringUtils" %> +<%@ page import="org.apache.tajo.ResourceProtos.FetcherHistoryProto" %> <%@ page import="org.apache.tajo.TaskAttemptId" %> -<%@ page import="org.apache.tajo.ipc.TajoWorkerProtocol" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.TajoIdUtils" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> -<%@ page import="org.apache.tajo.worker.*" %> +<%@ page import="org.apache.tajo.worker.Fetcher" %> +<%@ page import="org.apache.tajo.worker.TajoWorker" %> +<%@ page import="org.apache.tajo.worker.Task" %> +<%@ page import="org.apache.tajo.worker.TaskHistory" %> <%@ page import="java.text.SimpleDateFormat" %> <%@ page import="java.util.List" %> @@ -96,7 +99,7 @@ int index = 1; int pageSize = 1000; //TODO pagination - List fetcherHistories = taskHistory.getFetcherHistories(); + List fetcherHistories = taskHistory.getFetcherHistories(); if (fetcherHistories.size() > 0) { %> @@ -112,7 +115,7 @@ # Messages <% - for (TajoWorkerProtocol.FetcherHistoryProto eachFetcher : fetcherHistories) { + for (FetcherHistoryProto eachFetcher : fetcherHistories) { %> <%=index%> diff --git a/tajo-core/src/main/resources/webapps/worker/taskhistory.jsp b/tajo-core/src/main/resources/webapps/worker/taskhistory.jsp index b7774e88a3..5dd52d3d2c 100644 --- a/tajo-core/src/main/resources/webapps/worker/taskhistory.jsp +++ b/tajo-core/src/main/resources/webapps/worker/taskhistory.jsp @@ -19,7 +19,7 @@ %> <%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%> -<%@ page import="org.apache.tajo.ipc.TajoWorkerProtocol" %> +<%@ page import="org.apache.tajo.ResourceProtos.FetcherHistoryProto" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> <%@ page import="org.apache.tajo.worker.*" %> @@ -86,13 +86,13 @@ int index = 1; int pageSize = 1000; //TODO pagination - List fetcherHistories = taskHistory.getFetcherHistories(); + List fetcherHistories = taskHistory.getFetcherHistories(); if (fetcherHistories.size() > 0) { %> <% - for (TajoWorkerProtocol.FetcherHistoryProto eachFetcher : fetcherHistories) { + for (FetcherHistoryProto eachFetcher : fetcherHistories) { %> diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryState.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryState.java index a822e4251d..d2241b49e2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryState.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryState.java @@ -18,28 +18,36 @@ package org.apache.tajo.querymaster; +import net.jcip.annotations.NotThreadSafe; import org.apache.tajo.*; +import org.apache.tajo.client.QueryStatus; import org.apache.tajo.client.TajoClient; +import org.apache.tajo.client.TajoClientUtil; import org.apache.tajo.ipc.ClientProtos; -import org.apache.tajo.master.QueryManager; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; +import static org.junit.Assert.*; @Category(IntegrationTest.class) +@NotThreadSafe public class TestQueryState { private static TajoTestingCluster cluster; private static TajoClient client; @BeforeClass - public static void setUp() throws Exception { + public static void setUpClass() throws Exception { cluster = TpchTestBase.getInstance().getTestingCluster(); client = cluster.newTajoClient(); } + @AfterClass + public static void tearDownClass() { + client.close(); + } + @Test(timeout = 10000) public void testSucceededState() throws Exception { String queryStr = "select l_orderkey from lineitem group by l_orderkey order by l_orderkey"; @@ -61,16 +69,21 @@ public void testSucceededState() throws Exception { ClientProtos.SubmitQueryResponse res = client.executeQuery(queryStr); QueryId queryId = new QueryId(res.getQueryId()); - cluster.waitForQuerySubmitted(queryId); + + QueryStatus queryState = client.getQueryStatus(queryId); + while (!TajoClientUtil.isQueryComplete(queryState.getState())) { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + fail("Query state : " + queryState); + } + queryState = client.getQueryStatus(queryId); + } QueryMasterTask qmt = cluster.getQueryMasterTask(queryId); Query query = qmt.getQuery(); - // wait for query complete - cluster.waitForQueryState(query, TajoProtos.QueryState.QUERY_SUCCEEDED, 100); - assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, qmt.getState()); - assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, query.getSynchronizedState()); assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, query.getState()); @@ -80,13 +93,6 @@ public void testSucceededState() throws Exception { assertEquals(StageState.SUCCEEDED, stage.getState()); } - /* wait for heartbeat from QueryMaster */ - QueryManager queryManager = cluster.getMaster().getContext().getQueryJobManager(); - for (; ; ) { - if (queryManager.getFinishedQuery(queryId) != null) break; - else Thread.sleep(100); - } - /* get status from TajoMaster */ assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, client.getQueryStatus(queryId).getState()); } From cae89b7301fb0475491e2bbfe96f60bbdec5f71c Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 17 Jul 2015 12:56:56 +0900 Subject: [PATCH 70/80] print active threads --- tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java | 2 +- tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java index a323f252f6..995d448712 100644 --- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java @@ -231,7 +231,7 @@ public void printTestName() { /* protect a travis stalled build */ System.out.println("Run: " + name.getMethodName() + " Used memory: " + ((Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) - / (1024 * 1024)) + "MBytes"); + / (1024 * 1024)) + " MBytes, Active Threads:" + Thread.activeCount()); } public QueryTestCaseBase() { diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 8cb9e29d42..7be0cab089 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -128,7 +128,7 @@ void initPropertiesAndConfigs() { conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES.varname, 4); conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 2000); - conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 3);; + conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM.varname, 3); conf.setInt(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM.varname, 2); // Client API RPC From b90506818f608d43e37b11e0d41426a22bed99d2 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 17 Jul 2015 13:22:53 +0900 Subject: [PATCH 71/80] reduce threads for testing --- .../java/org/apache/tajo/worker/TestNodeResourceManager.java | 1 + .../java/org/apache/tajo/worker/TestNodeStatusUpdater.java | 3 +++ .../src/test/java/org/apache/tajo/worker/TestTaskExecutor.java | 2 ++ .../src/test/java/org/apache/tajo/worker/TestTaskManager.java | 1 + 4 files changed, 7 insertions(+) diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index b7c3967d7f..7539beb146 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -67,6 +67,7 @@ public void setup() { taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS, 4); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); + conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM, 2); dispatcher = new AsyncDispatcher(); taskDispatcher = new AsyncDispatcher(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index db8aa737e3..707aa496f4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -51,6 +51,9 @@ public class TestNodeStatusUpdater { public void setup() { conf = new TajoConf(); conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 2); + conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM, 2); + workerContext = new MockWorkerContext() { WorkerConnectionInfo workerConnectionInfo; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index da596093c2..9b6af682c1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -62,6 +62,8 @@ public class TestTaskExecutor { public void setup() { conf = new TajoConf(); conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 2); + conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM, 2); dispatcher = new AsyncDispatcher(); taskDispatcher = new AsyncDispatcher(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index 8d468c19a3..630f3f7b6d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -70,6 +70,7 @@ public void setup() { taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS, 4); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); + conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM, 2); dispatcher = new AsyncDispatcher(); taskDispatcher = new AsyncDispatcher(); From 655ed744c9992d8dbfebf7bc82869639e7981445 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 17 Jul 2015 17:41:14 +0900 Subject: [PATCH 72/80] rename worker to node --- .../tajo/master/QueryCoordinatorService.java | 8 +- .../apache/tajo/master/QueryInProgress.java | 11 +- .../org/apache/tajo/master/QueryManager.java | 3 +- .../tajo/master/TajoMasterClientService.java | 22 +-- .../NonForwardQueryResultSystemScanner.java | 68 ++++----- .../rm/{WorkerEvent.java => NodeEvent.java} | 8 +- ...orkerEventType.java => NodeEventType.java} | 6 +- ...onitor.java => NodeLivelinessMonitor.java} | 12 +- ...nectEvent.java => NodeReconnectEvent.java} | 18 +-- .../rm/{WorkerState.java => NodeState.java} | 14 +- .../rm/{Worker.java => NodeStatus.java} | 138 +++++++++--------- ...rStatusEvent.java => NodeStatusEvent.java} | 10 +- .../apache/tajo/master/rm/TajoRMContext.java | 20 +-- .../tajo/master/rm/TajoResourceManager.java | 26 ++-- .../tajo/master/rm/TajoResourceTracker.java | 82 +++++------ .../scheduler/AbstractQueryScheduler.java | 3 + .../master/scheduler/SimpleScheduler.java | 40 ++--- .../WorkerResourceMetricsGaugeSet.java | 16 +- .../querymaster/DefaultTaskScheduler.java | 4 +- .../tajo/querymaster/QueryMasterTask.java | 2 +- .../tajo/ws/rs/resources/ClusterResource.java | 8 +- .../tajo/ws/rs/responses/WorkerResponse.java | 13 +- .../main/resources/webapps/admin/cluster.jsp | 89 ++++++----- .../main/resources/webapps/admin/index.jsp | 19 +-- .../main/resources/webapps/admin/query.jsp | 8 +- .../resources/webapps/admin/querytasks.jsp | 10 +- .../master/scheduler/TestSimpleScheduler.java | 18 +-- 27 files changed, 338 insertions(+), 338 deletions(-) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{WorkerEvent.java => NodeEvent.java} (81%) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{WorkerEventType.java => NodeEventType.java} (88%) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{WorkerLivelinessMonitor.java => NodeLivelinessMonitor.java} (82%) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{WorkerReconnectEvent.java => NodeReconnectEvent.java} (69%) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{WorkerState.java => NodeState.java} (80%) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{Worker.java => NodeStatus.java} (62%) rename tajo-core/src/main/java/org/apache/tajo/master/rm/{WorkerStatusEvent.java => NodeStatusEvent.java} (81%) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java index 6457e6106b..3b04fc5b09 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryCoordinatorService.java @@ -28,7 +28,7 @@ import org.apache.tajo.ipc.QueryCoordinatorProtocol; import org.apache.tajo.ipc.QueryCoordinatorProtocol.*; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.master.rm.NodeStatus; import org.apache.tajo.master.scheduler.event.ResourceReserveSchedulerEvent; import org.apache.tajo.rpc.AsyncRpcServer; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; @@ -132,10 +132,10 @@ public void getAllWorkers(RpcController controller, PrimitiveProtos.NullProto re RpcCallback done) { WorkerConnectionsResponse.Builder builder = WorkerConnectionsResponse.newBuilder(); - Collection workers = context.getResourceManager().getRMContext().getWorkers().values(); + Collection nodeStatuses = context.getResourceManager().getRMContext().getNodes().values(); - for(Worker worker: workers) { - builder.addWorker(worker.getConnectionInfo().getProto()); + for(NodeStatus nodeStatus : nodeStatuses) { + builder.addWorker(nodeStatus.getConnectionInfo().getProto()); } done.run(builder.build()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java index 0762589c27..e22663a11b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryInProgress.java @@ -42,7 +42,6 @@ import java.net.ConnectException; import java.net.InetSocketAddress; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -56,9 +55,9 @@ public class QueryInProgress { private LogicalRootNode plan; - private volatile boolean querySubmitted = false; + private volatile boolean querySubmitted; - private AtomicBoolean isStopped = new AtomicBoolean(false); + private volatile boolean isStopped; private QueryInfo queryInfo; @@ -109,8 +108,10 @@ public void kill() { } public void stopProgress() { - if(isStopped.getAndSet(true)) { + if (isStopped) { return; + } else { + isStopped = true; } LOG.info("========================================================="); @@ -143,7 +144,7 @@ protected boolean allocateToQueryMaster(AllocationResourceProto allocation) { try { TajoResourceManager resourceManager = masterContext.getResourceManager(); WorkerConnectionInfo connectionInfo = - resourceManager.getRMContext().getWorkers().get(allocation.getWorkerId()).getConnectionInfo(); + resourceManager.getRMContext().getNodes().get(allocation.getWorkerId()).getConnectionInfo(); try { if(queryMasterRpcClient == null) { connectQueryMaster(connectionInfo); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java index d8c1ea1acd..b4ed5fd3f1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryManager.java @@ -56,6 +56,7 @@ */ public class QueryManager extends CompositeService { private static final Log LOG = LogFactory.getLog(QueryManager.class.getName()); + private static final String EMPTY_QM_HOSTNAME = ""; // TajoMaster Context private final TajoMaster.MasterContext masterContext; @@ -174,7 +175,7 @@ public QueryInfo scheduleQuery(Session session, QueryContext queryContext, Strin QueryInProgress queryInProgress = new QueryInProgress(masterContext, session, queryContext, queryId, sql, jsonExpr, plan); - queryInProgress.getQueryInfo().setQueryMaster(""); + queryInProgress.getQueryInfo().setQueryMaster(EMPTY_QM_HOSTNAME); submittedQueries.put(queryInProgress.getQueryId(), queryInProgress); //TODO implement scheduler queue QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo("default", queryContext.getUser(), diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java index 64e7a73d75..e0332d5d99 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java @@ -47,7 +47,7 @@ import org.apache.tajo.master.TajoMaster.MasterContext; import org.apache.tajo.master.exec.NonForwardQueryResultFileScanner; import org.apache.tajo.master.exec.NonForwardQueryResultScanner; -import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.master.rm.NodeStatus; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.logical.PartitionedTableScanNode; import org.apache.tajo.plan.logical.ScanNode; @@ -633,20 +633,20 @@ public GetClusterInfoResponse getClusterInfo(RpcController controller, context.getSessionManager().touch(request.getSessionId().getId()); GetClusterInfoResponse.Builder builder= GetClusterInfoResponse.newBuilder(); - List workers = new ArrayList(context.getResourceManager().getRMContext().getWorkers().values()); - Collections.sort(workers); + List nodeStatusList = new ArrayList(context.getResourceManager().getRMContext().getNodes().values()); + Collections.sort(nodeStatusList); WorkerResourceInfo.Builder workerBuilder = WorkerResourceInfo.newBuilder(); - for(Worker worker: workers) { - workerBuilder.setConnectionInfo(worker.getConnectionInfo().getProto()); - workerBuilder.setAvailableResource(worker.getAvailableResource().getProto()); - workerBuilder.setTotalResource(worker.getTotalResourceCapability().getProto()); + for(NodeStatus nodeStatus : nodeStatusList) { + workerBuilder.setConnectionInfo(nodeStatus.getConnectionInfo().getProto()); + workerBuilder.setAvailableResource(nodeStatus.getAvailableResource().getProto()); + workerBuilder.setTotalResource(nodeStatus.getTotalResourceCapability().getProto()); - workerBuilder.setLastHeartbeat(worker.getLastHeartbeatTime()); - workerBuilder.setWorkerStatus(worker.getState().toString()); - workerBuilder.setNumRunningTasks(worker.getNumRunningTasks()); - workerBuilder.setNumQueryMasterTasks(worker.getNumRunningQueryMaster()); + workerBuilder.setLastHeartbeat(nodeStatus.getLastHeartbeatTime()); + workerBuilder.setWorkerStatus(nodeStatus.getState().toString()); + workerBuilder.setNumRunningTasks(nodeStatus.getNumRunningTasks()); + workerBuilder.setNumQueryMasterTasks(nodeStatus.getNumRunningQueryMaster()); builder.addWorkerList(workerBuilder.build()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java index 173c270aed..58837a0623 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java @@ -41,7 +41,7 @@ import org.apache.tajo.engine.planner.physical.PhysicalExec; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.master.TajoMaster.MasterContext; -import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.master.rm.NodeStatus; import org.apache.tajo.plan.InvalidQueryException; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.PlanningException; @@ -421,7 +421,7 @@ private List getAllPartitions(Schema outSchema) { return tuples; } - private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { + private Tuple getQueryMasterTuple(Schema outSchema, NodeStatus aNodeStatus) { List columns = outSchema.getRootColumns(); Tuple aTuple = new VTuple(outSchema.size()); @@ -429,27 +429,27 @@ private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { Column column = columns.get(fieldId); if ("host".equalsIgnoreCase(column.getSimpleName())) { - if (aWorker.getConnectionInfo() != null && aWorker.getConnectionInfo().getHost() != null) { - aTuple.put(fieldId, DatumFactory.createText(aWorker.getConnectionInfo().getHost())); + if (aNodeStatus.getConnectionInfo() != null && aNodeStatus.getConnectionInfo().getHost() != null) { + aTuple.put(fieldId, DatumFactory.createText(aNodeStatus.getConnectionInfo().getHost())); } else { aTuple.put(fieldId, DatumFactory.createNullDatum()); } } else if ("port".equalsIgnoreCase(column.getSimpleName())) { - if (aWorker.getConnectionInfo() != null) { - aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getConnectionInfo().getQueryMasterPort())); + if (aNodeStatus.getConnectionInfo() != null) { + aTuple.put(fieldId, DatumFactory.createInt4(aNodeStatus.getConnectionInfo().getQueryMasterPort())); } else { aTuple.put(fieldId, DatumFactory.createNullDatum()); } } else if ("type".equalsIgnoreCase(column.getSimpleName())) { aTuple.put(fieldId, DatumFactory.createText("QueryMaster")); } else if ("status".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createText(aWorker.getState().toString())); - } else if ("RUNNING".equalsIgnoreCase(aWorker.getState().toString())) { + aTuple.put(fieldId, DatumFactory.createText(aNodeStatus.getState().toString())); + } else if ("RUNNING".equalsIgnoreCase(aNodeStatus.getState().toString())) { if ("running_tasks".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getNumRunningQueryMaster())); + aTuple.put(fieldId, DatumFactory.createInt4(aNodeStatus.getNumRunningQueryMaster())); } else if ("last_heartbeat_ts".equalsIgnoreCase(column.getSimpleName())) { - if (aWorker.getLastHeartbeatTime() > 0) { - aTuple.put(fieldId, DatumFactory.createTimestmpDatumWithJavaMillis(aWorker.getLastHeartbeatTime())); + if (aNodeStatus.getLastHeartbeatTime() > 0) { + aTuple.put(fieldId, DatumFactory.createTimestmpDatumWithJavaMillis(aNodeStatus.getLastHeartbeatTime())); } else { aTuple.put(fieldId, DatumFactory.createNullDatum()); } @@ -462,33 +462,33 @@ private Tuple getQueryMasterTuple(Schema outSchema, Worker aWorker) { return aTuple; } - private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) { + private Tuple getWorkerTuple(Schema outSchema, NodeStatus aNodeStatus) { List columns = outSchema.getRootColumns(); Tuple aTuple = new VTuple(outSchema.size()); - NodeResource total = aWorker.getTotalResourceCapability(); - NodeResource used = NodeResources.subtract(total, aWorker.getAvailableResource()); + NodeResource total = aNodeStatus.getTotalResourceCapability(); + NodeResource used = NodeResources.subtract(total, aNodeStatus.getAvailableResource()); for (int fieldId = 0; fieldId < columns.size(); fieldId++) { Column column = columns.get(fieldId); if ("host".equalsIgnoreCase(column.getSimpleName())) { - if (aWorker.getConnectionInfo() != null && aWorker.getConnectionInfo().getHost() != null) { - aTuple.put(fieldId, DatumFactory.createText(aWorker.getConnectionInfo().getHost())); + if (aNodeStatus.getConnectionInfo() != null && aNodeStatus.getConnectionInfo().getHost() != null) { + aTuple.put(fieldId, DatumFactory.createText(aNodeStatus.getConnectionInfo().getHost())); } else { aTuple.put(fieldId, DatumFactory.createNullDatum()); } } else if ("port".equalsIgnoreCase(column.getSimpleName())) { - if (aWorker.getConnectionInfo() != null) { - aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getConnectionInfo().getPeerRpcPort())); + if (aNodeStatus.getConnectionInfo() != null) { + aTuple.put(fieldId, DatumFactory.createInt4(aNodeStatus.getConnectionInfo().getPeerRpcPort())); } else { aTuple.put(fieldId, DatumFactory.createNullDatum()); } } else if ("type".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createText("Worker")); + aTuple.put(fieldId, DatumFactory.createText("NodeStatus")); } else if ("status".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createText(aWorker.getState().toString())); - } else if ("RUNNING".equalsIgnoreCase(aWorker.getState().toString())) { + aTuple.put(fieldId, DatumFactory.createText(aNodeStatus.getState().toString())); + } else if ("RUNNING".equalsIgnoreCase(aNodeStatus.getState().toString())) { if ("total_cpu".equalsIgnoreCase(column.getSimpleName())) { aTuple.put(fieldId, DatumFactory.createInt4(total.getVirtualCores())); } else if ("used_mem".equalsIgnoreCase(column.getSimpleName())) { @@ -500,10 +500,10 @@ private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) { } else if ("total_disk".equalsIgnoreCase(column.getSimpleName())) { aTuple.put(fieldId, DatumFactory.createInt4(total.getDisks())); } else if ("running_tasks".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createInt4(aWorker.getNumRunningTasks())); + aTuple.put(fieldId, DatumFactory.createInt4(aNodeStatus.getNumRunningTasks())); } else if ("last_heartbeat_ts".equalsIgnoreCase(column.getSimpleName())) { - if (aWorker.getLastHeartbeatTime() > 0) { - aTuple.put(fieldId, DatumFactory.createTimestmpDatumWithJavaMillis(aWorker.getLastHeartbeatTime())); + if (aNodeStatus.getLastHeartbeatTime() > 0) { + aTuple.put(fieldId, DatumFactory.createTimestmpDatumWithJavaMillis(aNodeStatus.getLastHeartbeatTime())); } else { aTuple.put(fieldId, DatumFactory.createNullDatum()); } @@ -518,23 +518,23 @@ private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) { } private List getClusterInfo(Schema outSchema) { - Map workerMap = masterContext.getResourceManager().getWorkers(); + Map workerMap = masterContext.getResourceManager().getNodes(); List tuples; - List queryMasterList = new ArrayList(); - List workerList = new ArrayList(); + List queryMasterList = new ArrayList(); + List nodeStatusList = new ArrayList(); - for (Worker aWorker: workerMap.values()) { - queryMasterList.add(aWorker); - workerList.add(aWorker); + for (NodeStatus aNodeStatus : workerMap.values()) { + queryMasterList.add(aNodeStatus); + nodeStatusList.add(aNodeStatus); } - tuples = new ArrayList(queryMasterList.size() + workerList.size()); - for (Worker queryMaster: queryMasterList) { + tuples = new ArrayList(queryMasterList.size() + nodeStatusList.size()); + for (NodeStatus queryMaster: queryMasterList) { tuples.add(getQueryMasterTuple(outSchema, queryMaster)); } - for (Worker worker: workerList) { - tuples.add(getWorkerTuple(outSchema, worker)); + for (NodeStatus nodeStatus : nodeStatusList) { + tuples.add(getWorkerTuple(outSchema, nodeStatus)); } return tuples; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeEvent.java similarity index 81% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEvent.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/NodeEvent.java index c208990971..430dc86160 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeEvent.java @@ -21,13 +21,13 @@ import org.apache.hadoop.yarn.event.AbstractEvent; /** - * WorkerEvent describes all kinds of events which sent to {@link Worker}. + * NodeEvent describes all kinds of events which sent to {@link NodeStatus}. */ -public class WorkerEvent extends AbstractEvent { +public class NodeEvent extends AbstractEvent { private final int workerId; - public WorkerEvent(int workerId, WorkerEventType workerEventType) { - super(workerEventType); + public NodeEvent(int workerId, NodeEventType nodeEventType) { + super(nodeEventType); this.workerId = workerId; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeEventType.java similarity index 88% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEventType.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/NodeEventType.java index 0c976543a5..64de5d43cd 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerEventType.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeEventType.java @@ -18,13 +18,13 @@ package org.apache.tajo.master.rm; -public enum WorkerEventType { +public enum NodeEventType { - /** Source : {@link TajoResourceTracker}, Destination: {@link Worker} */ + /** Source : {@link TajoResourceTracker}, Destination: {@link NodeStatus} */ STARTED, STATE_UPDATE, RECONNECTED, - /** Source : {@link WorkerLivelinessMonitor}, Destination: {@link Worker} */ + /** Source : {@link NodeLivelinessMonitor}, Destination: {@link NodeStatus} */ EXPIRE } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeLivelinessMonitor.java similarity index 82% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/NodeLivelinessMonitor.java index 7638ec8255..55a24fcda8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerLivelinessMonitor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeLivelinessMonitor.java @@ -27,15 +27,15 @@ import org.apache.tajo.conf.TajoConf; /** - * It periodically checks the latest heartbeat time of {@link Worker}. - * If the latest heartbeat time is expired, it produces EXPIRE event to a corresponding {@link Worker}. + * It periodically checks the latest heartbeat time of {@link NodeStatus}. + * If the latest heartbeat time is expired, it produces EXPIRE event to a corresponding {@link NodeStatus}. */ -public class WorkerLivelinessMonitor extends AbstractLivelinessMonitor { +public class NodeLivelinessMonitor extends AbstractLivelinessMonitor { private EventHandler dispatcher; - public WorkerLivelinessMonitor(Dispatcher d) { - super(WorkerLivelinessMonitor.class.getSimpleName(), new SystemClock()); + public NodeLivelinessMonitor(Dispatcher d) { + super(NodeLivelinessMonitor.class.getSimpleName(), new SystemClock()); this.dispatcher = d.getEventHandler(); } @@ -51,6 +51,6 @@ public void serviceInit(Configuration conf) throws Exception { @Override protected void expire(Integer id) { - dispatcher.handle(new WorkerEvent(id, WorkerEventType.EXPIRE)); + dispatcher.handle(new NodeEvent(id, NodeEventType.EXPIRE)); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerReconnectEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeReconnectEvent.java similarity index 69% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerReconnectEvent.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/NodeReconnectEvent.java index 3828b6aa90..e9ba0777d6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerReconnectEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeReconnectEvent.java @@ -19,17 +19,17 @@ package org.apache.tajo.master.rm; /** - * {@link TajoResourceTracker} produces this event, and it's destination is {@link Worker}. - * This event occurs only when an inactive worker sends a ping again. + * {@link TajoResourceTracker} produces this event, and it's destination is {@link NodeStatus}. + * This event occurs only when an inactive nodeStatus sends a ping again. */ -public class WorkerReconnectEvent extends WorkerEvent { - private final Worker worker; - public WorkerReconnectEvent(int workerId, Worker worker) { - super(workerId, WorkerEventType.RECONNECTED); - this.worker = worker; +public class NodeReconnectEvent extends NodeEvent { + private final NodeStatus nodeStatus; + public NodeReconnectEvent(int workerId, NodeStatus nodeStatus) { + super(workerId, NodeEventType.RECONNECTED); + this.nodeStatus = nodeStatus; } - public Worker getWorker() { - return worker; + public NodeStatus getNodeStatus() { + return nodeStatus; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerState.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeState.java similarity index 80% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerState.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/NodeState.java index a941008bae..89f601d56c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerState.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeState.java @@ -19,22 +19,22 @@ package org.apache.tajo.master.rm; /** - * It presents the states of {@link Worker}. + * It presents the states of {@link NodeStatus}. */ -public enum WorkerState { - /** New worker */ +public enum NodeState { + /** New node */ NEW, - /** Running worker */ + /** Running node */ RUNNING, - /** Worker is unhealthy */ + /** Node is unhealthy */ UNHEALTHY, - /** worker is out of service */ + /** Node is out of service */ DECOMMISSIONED, - /** worker has not sent a heartbeat for some configured time threshold */ + /** Node has not sent a heartbeat for some configured time threshold */ LOST; @SuppressWarnings("unused") diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatus.java similarity index 62% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatus.java index 3660689513..fc1834b9c2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/Worker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatus.java @@ -33,11 +33,11 @@ import java.util.concurrent.atomic.AtomicLongFieldUpdater; /** - * It contains resource and various information for a worker. + * It contains resource and various information for a node. */ -public class Worker implements EventHandler, Comparable { +public class NodeStatus implements EventHandler, Comparable { /** class logger */ - private static final Log LOG = LogFactory.getLog(Worker.class); + private static final Log LOG = LogFactory.getLog(NodeStatus.class); /** context of {@link TajoResourceManager} */ private final TajoRMContext rmContext; @@ -54,14 +54,14 @@ public class Worker implements EventHandler, Comparable { private static AtomicIntegerFieldUpdater RUNNING_QM_UPDATER; static { - HEARTBEAT_TIME_UPDATER = PlatformDependent.newAtomicLongFieldUpdater(Worker.class, "lastHeartbeatTime"); + HEARTBEAT_TIME_UPDATER = PlatformDependent.newAtomicLongFieldUpdater(NodeStatus.class, "lastHeartbeatTime"); if (HEARTBEAT_TIME_UPDATER == null) { - HEARTBEAT_TIME_UPDATER = AtomicLongFieldUpdater.newUpdater(Worker.class, "lastHeartbeatTime"); - RUNNING_TASK_UPDATER = AtomicIntegerFieldUpdater.newUpdater(Worker.class, "numRunningTasks"); - RUNNING_QM_UPDATER = AtomicIntegerFieldUpdater.newUpdater(Worker.class, "numRunningQueryMaster"); + HEARTBEAT_TIME_UPDATER = AtomicLongFieldUpdater.newUpdater(NodeStatus.class, "lastHeartbeatTime"); + RUNNING_TASK_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeStatus.class, "numRunningTasks"); + RUNNING_QM_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeStatus.class, "numRunningQueryMaster"); } else { - RUNNING_TASK_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(Worker.class, "numRunningTasks"); - RUNNING_QM_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(Worker.class, "numRunningQueryMaster"); + RUNNING_TASK_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeStatus.class, "numRunningTasks"); + RUNNING_QM_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeStatus.class, "numRunningQueryMaster"); } } @@ -71,52 +71,52 @@ public class Worker implements EventHandler, Comparable { /** Total resources on the node. */ private final NodeResource totalResourceCapability; - /** Worker connection information */ + /** Node connection information */ private WorkerConnectionInfo connectionInfo; private static final ReconnectNodeTransition RECONNECT_NODE_TRANSITION = new ReconnectNodeTransition(); private static final StatusUpdateTransition STATUS_UPDATE_TRANSITION = new StatusUpdateTransition(); - private static final StateMachineFactory stateMachineFactory - = new StateMachineFactory(WorkerState.NEW) + private static final StateMachineFactory stateMachineFactory + = new StateMachineFactory(NodeState.NEW) // Transition from NEW - .addTransition(WorkerState.NEW, WorkerState.RUNNING, - WorkerEventType.STARTED, + .addTransition(NodeState.NEW, NodeState.RUNNING, + NodeEventType.STARTED, new AddNodeTransition()) // Transition from RUNNING - .addTransition(WorkerState.RUNNING, EnumSet.of(WorkerState.RUNNING, WorkerState.UNHEALTHY), - WorkerEventType.STATE_UPDATE, + .addTransition(NodeState.RUNNING, EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY), + NodeEventType.STATE_UPDATE, STATUS_UPDATE_TRANSITION) - .addTransition(WorkerState.RUNNING, WorkerState.LOST, - WorkerEventType.EXPIRE, - new DeactivateNodeTransition(WorkerState.LOST)) - .addTransition(WorkerState.RUNNING, WorkerState.RUNNING, - WorkerEventType.RECONNECTED, + .addTransition(NodeState.RUNNING, NodeState.LOST, + NodeEventType.EXPIRE, + new DeactivateNodeTransition(NodeState.LOST)) + .addTransition(NodeState.RUNNING, NodeState.RUNNING, + NodeEventType.RECONNECTED, RECONNECT_NODE_TRANSITION) // Transitions from UNHEALTHY state - .addTransition(WorkerState.UNHEALTHY, EnumSet.of(WorkerState.RUNNING, WorkerState.UNHEALTHY), - WorkerEventType.STATE_UPDATE, + .addTransition(NodeState.UNHEALTHY, EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY), + NodeEventType.STATE_UPDATE, STATUS_UPDATE_TRANSITION) - .addTransition(WorkerState.UNHEALTHY, WorkerState.LOST, - WorkerEventType.EXPIRE, - new DeactivateNodeTransition(WorkerState.LOST)) - .addTransition(WorkerState.UNHEALTHY, WorkerState.UNHEALTHY, - WorkerEventType.RECONNECTED, + .addTransition(NodeState.UNHEALTHY, NodeState.LOST, + NodeEventType.EXPIRE, + new DeactivateNodeTransition(NodeState.LOST)) + .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY, + NodeEventType.RECONNECTED, RECONNECT_NODE_TRANSITION); - private final StateMachine stateMachine = - stateMachineFactory.make(this, WorkerState.NEW); + private final StateMachine stateMachine = + stateMachineFactory.make(this, NodeState.NEW); - public Worker(TajoRMContext rmContext, NodeResource totalResourceCapability, WorkerConnectionInfo connectionInfo) { + public NodeStatus(TajoRMContext rmContext, NodeResource totalResourceCapability, WorkerConnectionInfo connectionInfo) { this.rmContext = rmContext; this.connectionInfo = connectionInfo; @@ -159,9 +159,9 @@ public long getLastHeartbeatTime() { /** * - * @return the current state of worker + * @return the current state of node */ - public WorkerState getState() { + public NodeState getState() { return this.stateMachine.getCurrentState(); } @@ -184,7 +184,7 @@ public NodeResource getTotalResourceCapability() { } @Override - public int compareTo(Worker o) { + public int compareTo(NodeStatus o) { if(o == null) { return 1; } @@ -196,9 +196,9 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - Worker worker = (Worker) o; + NodeStatus nodeStatus = (NodeStatus) o; - if (connectionInfo != null ? !connectionInfo.equals(worker.connectionInfo) : worker.connectionInfo != null) + if (connectionInfo != null ? !connectionInfo.equals(nodeStatus.connectionInfo) : nodeStatus.connectionInfo != null) return false; return true; } @@ -210,29 +210,29 @@ public int hashCode() { return result; } - public static class AddNodeTransition implements SingleArcTransition { + public static class AddNodeTransition implements SingleArcTransition { @Override - public void transition(Worker worker, WorkerEvent workerEvent) { + public void transition(NodeStatus nodeStatus, NodeEvent nodeEvent) { - worker.rmContext.getQueryMasterWorker().add(worker.getWorkerId()); - LOG.info("Worker with " + worker.getTotalResourceCapability() + " is joined to Tajo cluster"); + nodeStatus.rmContext.getQueryMasterWorker().add(nodeStatus.getWorkerId()); + LOG.info("Node with " + nodeStatus.getTotalResourceCapability() + " is joined to Tajo cluster"); } } public static class StatusUpdateTransition implements - MultipleArcTransition { + MultipleArcTransition { @Override - public WorkerState transition(Worker worker, WorkerEvent event) { + public NodeState transition(NodeStatus nodeStatus, NodeEvent event) { - WorkerStatusEvent statusEvent = TUtil.checkTypeAndGet(event, WorkerStatusEvent.class); - worker.updateStatus(statusEvent); + NodeStatusEvent statusEvent = TUtil.checkTypeAndGet(event, NodeStatusEvent.class); + nodeStatus.updateStatus(statusEvent); - return WorkerState.RUNNING; + return NodeState.RUNNING; } } - private void updateStatus(WorkerStatusEvent statusEvent) { + private void updateStatus(NodeStatusEvent statusEvent) { setLastHeartbeatTime(System.currentTimeMillis()); setNumRunningTasks(statusEvent.getRunningTaskNum()); setNumRunningQueryMaster(statusEvent.getRunningQMNum()); @@ -243,39 +243,39 @@ private void updateStatus(WorkerStatusEvent statusEvent) { } } - public static class DeactivateNodeTransition implements SingleArcTransition { - private final WorkerState finalState; + public static class DeactivateNodeTransition implements SingleArcTransition { + private final NodeState finalState; - public DeactivateNodeTransition(WorkerState finalState) { + public DeactivateNodeTransition(NodeState finalState) { this.finalState = finalState; } @Override - public void transition(Worker worker, WorkerEvent workerEvent) { + public void transition(NodeStatus nodeStatus, NodeEvent nodeEvent) { - worker.rmContext.getWorkers().remove(worker.getWorkerId()); - LOG.info("Deactivating Node " + worker.getWorkerId() + " as it is now " + finalState); - worker.rmContext.getInactiveWorkers().putIfAbsent(worker.getWorkerId(), worker); + nodeStatus.rmContext.getNodes().remove(nodeStatus.getWorkerId()); + LOG.info("Deactivating Node " + nodeStatus.getWorkerId() + " as it is now " + finalState); + nodeStatus.rmContext.getInactiveNodes().putIfAbsent(nodeStatus.getWorkerId(), nodeStatus); } } - public static class ReconnectNodeTransition implements SingleArcTransition { + public static class ReconnectNodeTransition implements SingleArcTransition { @Override - public void transition(Worker worker, WorkerEvent workerEvent) { + public void transition(NodeStatus nodeStatus, NodeEvent nodeEvent) { - WorkerReconnectEvent castedEvent = TUtil.checkTypeAndGet(workerEvent, WorkerReconnectEvent.class); - Worker newWorker = castedEvent.getWorker(); - worker.rmContext.getWorkers().put(castedEvent.getWorkerId(), newWorker); - worker.rmContext.getDispatcher().getEventHandler().handle( - new WorkerEvent(worker.getWorkerId(), WorkerEventType.STARTED)); + NodeReconnectEvent castedEvent = TUtil.checkTypeAndGet(nodeEvent, NodeReconnectEvent.class); + NodeStatus newNodeStatus = castedEvent.getNodeStatus(); + nodeStatus.rmContext.getNodes().put(castedEvent.getWorkerId(), newNodeStatus); + nodeStatus.rmContext.getDispatcher().getEventHandler().handle( + new NodeEvent(nodeStatus.getWorkerId(), NodeEventType.STARTED)); } } @Override - public void handle(WorkerEvent event) { + public void handle(NodeEvent event) { LOG.debug("Processing " + event.getWorkerId() + " of type " + event.getType()); - WorkerState oldState = getState(); + NodeState oldState = getState(); try { stateMachine.doTransition(event.getType(), event); } catch (InvalidStateTransitonException e) { @@ -284,7 +284,7 @@ public void handle(WorkerEvent event) { + ", oldState:" + oldState.name() + ", nextState:" + getState().name() , e); - LOG.error("Invalid event " + event.getType() + " on Worker " + getWorkerId()); + LOG.error("Invalid event " + event.getType() + " on NodeStatus " + getWorkerId()); } if (oldState != getState()) { LOG.info(getWorkerId() + " Node Transitioned from " + oldState + " to " + getState()); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatusEvent.java similarity index 81% rename from tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java rename to tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatusEvent.java index 11cf57e680..db2f167ea7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatusEvent.java @@ -23,17 +23,17 @@ /** * {@link TajoResourceTracker} produces this event, and its destination is - * {@link org.apache.tajo.master.rm.Worker.StatusUpdateTransition} of {@link Worker}. + * {@link NodeStatus.StatusUpdateTransition} of {@link NodeStatus}. */ -public class WorkerStatusEvent extends WorkerEvent { +public class NodeStatusEvent extends NodeEvent { private final int runningTaskNum; private final int runningQMNum; private final NodeResource available; private final NodeResource total; - public WorkerStatusEvent(int workerId, int runningTaskNum, int runningQMNum, - NodeResource available, @Nullable NodeResource total) { - super(workerId, WorkerEventType.STATE_UPDATE); + public NodeStatusEvent(int workerId, int runningTaskNum, int runningQMNum, + NodeResource available, @Nullable NodeResource total) { + super(workerId, NodeEventType.STATE_UPDATE); this.runningTaskNum = runningTaskNum; this.runningQMNum = runningQMNum; this.available = available; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java index 41aea78c81..9dfae7e093 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java @@ -33,11 +33,11 @@ public class TajoRMContext { final Dispatcher rmDispatcher; - /** map between workerIds and running workers */ - private final ConcurrentMap workers = Maps.newConcurrentMap(); + /** map between workerIds and running nodes */ + private final ConcurrentMap nodes = Maps.newConcurrentMap(); - /** map between workerIds and inactive workers */ - private final ConcurrentMap inactiveWorkers = Maps.newConcurrentMap(); + /** map between workerIds and inactive nodes */ + private final ConcurrentMap inactiveNodes = Maps.newConcurrentMap(); private final Set liveQueryMasterWorkerResources = Collections.newSetFromMap(new ConcurrentHashMap()); @@ -52,17 +52,17 @@ public Dispatcher getDispatcher() { } /** - * @return The Map for active workers + * @return The Map for active nodes */ - public ConcurrentMap getWorkers() { - return workers; + public ConcurrentMap getNodes() { + return nodes; } /** - * @return The Map for inactive workers + * @return The Map for inactive nodes */ - public ConcurrentMap getInactiveWorkers() { - return inactiveWorkers; + public ConcurrentMap getInactiveNodes() { + return inactiveNodes; } public Set getQueryMasterWorker() { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java index 8946a0f5ce..c91938a9c9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceManager.java @@ -57,9 +57,9 @@ public class TajoResourceManager extends CompositeService { private String queryIdSeed; /** - * Worker Liveliness monitor + * Node Liveliness monitor */ - private WorkerLivelinessMonitor workerLivelinessMonitor; + private NodeLivelinessMonitor nodeLivelinessMonitor; private TajoConf systemConf; private AbstractQueryScheduler scheduler; @@ -83,13 +83,13 @@ public void serviceInit(Configuration conf) throws Exception { this.queryIdSeed = String.valueOf(System.currentTimeMillis()); - this.workerLivelinessMonitor = new WorkerLivelinessMonitor(this.rmContext.getDispatcher()); - addIfService(this.workerLivelinessMonitor); + this.nodeLivelinessMonitor = new NodeLivelinessMonitor(this.rmContext.getDispatcher()); + addIfService(this.nodeLivelinessMonitor); // Register event handler for Workers - rmContext.getDispatcher().register(WorkerEventType.class, new WorkerEventDispatcher(rmContext)); + rmContext.getDispatcher().register(NodeEventType.class, new WorkerEventDispatcher(rmContext)); - resourceTracker = new TajoResourceTracker(this, workerLivelinessMonitor); + resourceTracker = new TajoResourceTracker(this, nodeLivelinessMonitor); addIfService(resourceTracker); String schedulerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_SCHEDULER_CLASS); @@ -118,7 +118,7 @@ protected synchronized AbstractQueryScheduler loadScheduler(String schedulerClas } @InterfaceAudience.Private - public static final class WorkerEventDispatcher implements EventHandler { + public static final class WorkerEventDispatcher implements EventHandler { private final TajoRMContext rmContext; @@ -127,9 +127,9 @@ public WorkerEventDispatcher(TajoRMContext rmContext) { } @Override - public void handle(WorkerEvent event) { + public void handle(NodeEvent event) { int workerId = event.getWorkerId(); - Worker node = this.rmContext.getWorkers().get(workerId); + NodeStatus node = this.rmContext.getNodes().get(workerId); if (node != null) { try { node.handle(event); @@ -141,12 +141,12 @@ public void handle(WorkerEvent event) { } - public Map getWorkers() { - return ImmutableMap.copyOf(rmContext.getWorkers()); + public Map getNodes() { + return ImmutableMap.copyOf(rmContext.getNodes()); } - public Map getInactiveWorkers() { - return ImmutableMap.copyOf(rmContext.getInactiveWorkers()); + public Map getInactiveNodes() { + return ImmutableMap.copyOf(rmContext.getInactiveNodes()); } public Collection getQueryMasters() { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index 8fbf0c403c..3e3e3b47bb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -40,18 +40,18 @@ import static org.apache.tajo.ResourceProtos.*; /** - * It receives pings that workers periodically send. The ping messages contains the worker resources and their statuses. - * From ping messages, {@link TajoResourceTracker} tracks the recent status of all workers. + * It receives pings that nodes periodically send. The ping messages contains the node resources and their statuses. + * From ping messages, {@link TajoResourceTracker} tracks the recent status of all nodes. * * In detail, it has two main roles as follows: * *
      *
    • Membership management for nodes which join to a Tajo cluster
    • *
        - *
      • Register - It receives the ping from a new worker. It registers the worker.
      • - *
      • Unregister - It unregisters a worker who does not send ping for some expiry time.
      • + *
      • Register - It receives the ping from a new node. It registers the node.
      • + *
      • Unregister - It unregisters a node who does not send ping for some expiry time.
      • *
          - *
        • Status Update - It updates the status of all participating workers
        • + *
        • Status Update - It updates the status of all participating nodes
        • *
        */ public class TajoResourceTracker extends AbstractService implements TajoResourceTrackerProtocolService.Interface { @@ -61,22 +61,22 @@ public class TajoResourceTracker extends AbstractService implements TajoResource private final TajoResourceManager manager; /** the context of TajoResourceManager */ private final TajoRMContext rmContext; - /** Liveliness monitor which checks ping expiry times of workers */ - private final WorkerLivelinessMonitor workerLivelinessMonitor; + /** Liveliness monitor which checks ping expiry times of nodes */ + private final NodeLivelinessMonitor nodeLivelinessMonitor; - /** RPC server for worker resource tracker */ + /** RPC server for node resource tracker */ private AsyncRpcServer server; - /** The bind address of RPC server of worker resource tracker */ + /** The bind address of RPC server of node resource tracker */ private InetSocketAddress bindAddress; - /** worker heartbeat interval in query running */ + /** node heartbeat interval in query running */ private int activeInterval; - public TajoResourceTracker(TajoResourceManager manager, WorkerLivelinessMonitor workerLivelinessMonitor) { + public TajoResourceTracker(TajoResourceManager manager, NodeLivelinessMonitor nodeLivelinessMonitor) { super(TajoResourceTracker.class.getSimpleName()); this.manager = manager; this.rmContext = manager.getRMContext(); - this.workerLivelinessMonitor = workerLivelinessMonitor; + this.nodeLivelinessMonitor = nodeLivelinessMonitor; } @Override @@ -110,8 +110,8 @@ public void serviceStop() throws Exception { super.serviceStop(); } - private static WorkerStatusEvent createStatusEvent(NodeHeartbeatRequest heartbeat) { - return new WorkerStatusEvent( + private static NodeStatusEvent createStatusEvent(NodeHeartbeatRequest heartbeat) { + return new NodeStatusEvent( heartbeat.getWorkerId(), heartbeat.getRunningTasks(), heartbeat.getRunningQueryMasters(), @@ -131,7 +131,7 @@ public void nodeHeartbeat( // get a workerId from the heartbeat int workerId = heartbeat.getWorkerId(); - if(rmContext.getWorkers().containsKey(workerId)) { // if worker is running + if(rmContext.getNodes().containsKey(workerId)) { // if node is running // status update rmContext.getDispatcher().getEventHandler().handle(createStatusEvent(heartbeat)); @@ -140,54 +140,54 @@ public void nodeHeartbeat( rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); // refresh ping - workerLivelinessMonitor.receivedPing(workerId); + nodeLivelinessMonitor.receivedPing(workerId); - } else if (rmContext.getInactiveWorkers().containsKey(workerId)) { // worker was inactive + } else if (rmContext.getInactiveNodes().containsKey(workerId)) { // node was inactive if (!heartbeat.hasConnectionInfo()) { // request membership to worker node responseCommand = ResponseCommand.MEMBERSHIP; } else { - // remove the inactive worker from the list of inactive workers. - Worker worker = rmContext.getInactiveWorkers().remove(workerId); - workerLivelinessMonitor.unregister(worker.getWorkerId()); + // remove the inactive nodeStatus from the list of inactive nodes. + NodeStatus nodeStatus = rmContext.getInactiveNodes().remove(workerId); + nodeLivelinessMonitor.unregister(nodeStatus.getWorkerId()); - // create new worker instance - Worker newWorker = createWorkerResource(heartbeat); - int newWorkerId = newWorker.getWorkerId(); - // add the new worker to the list of active workers - rmContext.getWorkers().putIfAbsent(newWorkerId, newWorker); + // create new nodeStatus instance + NodeStatus newNodeStatus = createNodeStatus(heartbeat); + int newWorkerId = newNodeStatus.getWorkerId(); + // add the new nodeStatus to the list of active nodes + rmContext.getNodes().putIfAbsent(newWorkerId, newNodeStatus); - // Transit the worker to RUNNING - rmContext.getDispatcher().getEventHandler().handle(new WorkerEvent(newWorkerId, WorkerEventType.STARTED)); - // register the worker to the liveliness monitor - workerLivelinessMonitor.register(newWorkerId); + // Transit the nodeStatus to RUNNING + rmContext.getDispatcher().getEventHandler().handle(new NodeEvent(newWorkerId, NodeEventType.STARTED)); + // register the nodeStatus to the liveliness monitor + nodeLivelinessMonitor.register(newWorkerId); rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); } - } else { // if new worker pings firstly + } else { // if new node pings firstly // The pings have not membership information if (!heartbeat.hasConnectionInfo()) { - // request membership to worker node + // request membership to node responseCommand = ResponseCommand.MEMBERSHIP; } else { - // create new worker instance - Worker newWorker = createWorkerResource(heartbeat); - Worker oldWorker = rmContext.getWorkers().putIfAbsent(workerId, newWorker); + // create new node instance + NodeStatus newNodeStatus = createNodeStatus(heartbeat); + NodeStatus oldNodeStatus = rmContext.getNodes().putIfAbsent(workerId, newNodeStatus); - if (oldWorker == null) { + if (oldNodeStatus == null) { // Transit the worker to RUNNING - rmContext.rmDispatcher.getEventHandler().handle(new WorkerEvent(workerId, WorkerEventType.STARTED)); + rmContext.rmDispatcher.getEventHandler().handle(new NodeEvent(workerId, NodeEventType.STARTED)); } else { LOG.info("Reconnect from the node at: " + workerId); - workerLivelinessMonitor.unregister(workerId); - rmContext.getDispatcher().getEventHandler().handle(new WorkerReconnectEvent(workerId, newWorker)); + nodeLivelinessMonitor.unregister(workerId); + rmContext.getDispatcher().getEventHandler().handle(new NodeReconnectEvent(workerId, newNodeStatus)); } - workerLivelinessMonitor.register(workerId); + nodeLivelinessMonitor.register(workerId); rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); } } @@ -199,8 +199,8 @@ public void nodeHeartbeat( } } - private Worker createWorkerResource(NodeHeartbeatRequest request) { - return new Worker(rmContext, new NodeResource(request.getTotalResource()), + private NodeStatus createNodeStatus(NodeHeartbeatRequest request) { + return new NodeStatus(rmContext, new NodeResource(request.getTotalResource()), new WorkerConnectionInfo(request.getConnectionInfo())); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java index 29694c6d86..1380417fe1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/AbstractQueryScheduler.java @@ -24,6 +24,9 @@ import org.apache.tajo.resource.NodeResources; import org.apache.tajo.resource.ResourceCalculator; +/** + * please refer to {@TajoResourceScheduler} for detailed information. + */ public abstract class AbstractQueryScheduler extends AbstractService implements TajoResourceScheduler { protected final NodeResource clusterResource; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 2201873d8c..6432f9875d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -31,7 +31,7 @@ import org.apache.tajo.master.TajoMaster; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.rm.TajoRMContext; -import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.master.rm.NodeStatus; import org.apache.tajo.master.scheduler.event.ResourceReserveSchedulerEvent; import org.apache.tajo.master.scheduler.event.SchedulerEvent; import org.apache.tajo.resource.DefaultResourceCalculator; @@ -93,9 +93,9 @@ private void initScheduler(TajoConf conf) { private void updateResource() { NodeResource resource = NodeResources.createResource(0); NodeResource totalResource = NodeResources.createResource(0); - for (Worker worker : getRMContext().getWorkers().values()) { - NodeResources.addTo(resource, worker.getAvailableResource()); - NodeResources.addTo(totalResource, worker.getTotalResourceCapability()); + for (NodeStatus nodeStatus : getRMContext().getNodes().values()) { + NodeResources.addTo(resource, nodeStatus.getAvailableResource()); + NodeResources.addTo(totalResource, nodeStatus.getTotalResourceCapability()); } @@ -144,11 +144,11 @@ private NodeResourceRequest createQMResourceRequest(QueryInfo queryInfo) { Set assignedQMNodes = Sets.newHashSet(assignedQueryMasterMap.values()); List idleNode = Lists.newArrayList(); - for (Worker worker : getRMContext().getWorkers().values()) { + for (NodeStatus nodeStatus : getRMContext().getNodes().values()) { //find idle node for QM - if (!assignedQMNodes.contains(worker.getWorkerId())) { - idleNode.add(worker.getWorkerId()); + if (!assignedQMNodes.contains(nodeStatus.getWorkerId())) { + idleNode.add(nodeStatus.getWorkerId()); } if (idleNode.size() > containers * 3) break; @@ -171,7 +171,7 @@ private NodeResourceRequest createQMResourceRequest(QueryInfo queryInfo) { @Override public int getNumClusterNodes() { - return rmContext.getWorkers().size(); + return rmContext.getNodes().size(); } @Override @@ -197,11 +197,11 @@ public int getNumClusterNodes() { // reserve resource in random workers if (reservedResources.size() < requiredContainers) { - LinkedList randomWorkers = new LinkedList(getRMContext().getWorkers().keySet()); - Collections.shuffle(randomWorkers); + LinkedList randomNodes = new LinkedList(getRMContext().getNodes().keySet()); + Collections.shuffle(randomNodes); reservedResources.addAll(reserveClusterResource( - randomWorkers, capacity, requiredContainers - reservedResources.size())); + randomNodes, capacity, requiredContainers - reservedResources.size())); } if (LOG.isDebugEnabled()) { @@ -223,21 +223,21 @@ private List reserveClusterResource(List worke while (iter.hasNext()) { int workerId = iter.next(); - Worker worker = getRMContext().getWorkers().get(workerId); - if (worker == null) { + NodeStatus nodeStatus = getRMContext().getNodes().get(workerId); + if (nodeStatus == null) { iter.remove(); - LOG.warn("Can't found the worker :" + workerId); + LOG.warn("Can't found the nodeStatus :" + workerId); continue; } else { - if (NodeResources.fitsIn(capacity, worker.getAvailableResource())) { + if (NodeResources.fitsIn(capacity, nodeStatus.getAvailableResource())) { NodeResources.subtractFrom(getClusterResource(), capacity); - NodeResources.subtractFrom(worker.getAvailableResource(), capacity); + NodeResources.subtractFrom(nodeStatus.getAvailableResource(), capacity); allocatedResources++; resourceBuilder.setResource(capacity.getProto()); resourceBuilder.setWorkerId(workerId); reservedResources.add(resourceBuilder.build()); } else { - // remove unavailable worker; + // remove unavailable nodeStatus; iter.remove(); } } @@ -302,8 +302,8 @@ public BlockingQueue getQueryQueue() { return queryQueue; } - private Worker getWorker(int workerId) { - return rmContext.getWorkers().get(workerId); + private NodeStatus getWorker(int workerId) { + return rmContext.getNodes().get(workerId); } protected TajoRMContext getRMContext() { @@ -312,7 +312,7 @@ protected TajoRMContext getRMContext() { public WorkerConnectionInfo getQueryMaster(QueryId queryId) { if (assignedQueryMasterMap.containsKey(queryId)) { - return rmContext.getWorkers().get(assignedQueryMasterMap.get(queryId)).getConnectionInfo(); + return rmContext.getNodes().get(assignedQueryMasterMap.get(queryId)).getConnectionInfo(); } return null; } diff --git a/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java index 229a80ae3b..200d689889 100644 --- a/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java +++ b/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java @@ -22,8 +22,8 @@ import com.codahale.metrics.Metric; import com.codahale.metrics.MetricSet; import org.apache.tajo.master.TajoMaster; -import org.apache.tajo.master.rm.Worker; -import org.apache.tajo.master.rm.WorkerState; +import org.apache.tajo.master.rm.NodeStatus; +import org.apache.tajo.master.rm.NodeState; import java.util.HashMap; import java.util.Map; @@ -40,31 +40,31 @@ public Map getMetrics() { metricsMap.put("totalWorkers", new Gauge() { @Override public Integer getValue() { - return tajoMasterContext.getResourceManager().getWorkers().size(); + return tajoMasterContext.getResourceManager().getNodes().size(); } }); metricsMap.put("liveWorkers", new Gauge() { @Override public Integer getValue() { - return getNumWorkers(WorkerState.RUNNING); + return getNumWorkers(NodeState.RUNNING); } }); metricsMap.put("deadWorkers", new Gauge() { @Override public Integer getValue() { - return getNumWorkers(WorkerState.LOST); + return getNumWorkers(NodeState.LOST); } }); return metricsMap; } - protected int getNumWorkers(WorkerState status) { + protected int getNumWorkers(NodeState status) { int numWorkers = 0; - for(Worker eachWorker: tajoMasterContext.getResourceManager().getWorkers().values()) { - if(eachWorker.getState() == status) { + for(NodeStatus eachNodeStatus : tajoMasterContext.getResourceManager().getNodes().values()) { + if(eachNodeStatus.getState() == status) { numWorkers++; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 34543e7e34..85a65bbd6c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -101,7 +101,7 @@ public void start() { LOG.info("Start TaskScheduler"); isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); - if (!isLeaf) { + if (isLeaf) { candidateWorkers.addAll(getWorkerIds(getLeafTaskHosts())); } else { //find assigned hosts for Non-Leaf locality in children executionBlock @@ -266,7 +266,7 @@ private Set getWorkerIds(Collection hosts){ protected LinkedList createTaskRequest() throws Exception { LinkedList taskRequestEvents = new LinkedList(); - int requestContainerNum = Math.max(remainingScheduledObjectNum(), 1); + int requestContainerNum = remainingScheduledObjectNum(); if (LOG.isDebugEnabled()) { LOG.debug("Try to schedule task resources: " + requestContainerNum); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java index bec16857be..10ed1ec70e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java @@ -284,7 +284,7 @@ protected void killTaskAttempt(int workerId, TaskAttemptId taskAttemptId) { new TaskFatalErrorEvent(taskAttemptId, "Can't kill task :" + taskAttemptId)); } } catch (Exception e) { - /* Worker RPC failure */ + /* NodeStatus RPC failure */ LOG.error(e.getMessage(), e); queryMasterContext.getEventHandler().handle(new TaskFatalErrorEvent(taskAttemptId, e.getMessage())); } diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/ClusterResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/ClusterResource.java index 349fa4cf5c..19d8b28685 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/ClusterResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/ClusterResource.java @@ -35,7 +35,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.tajo.master.TajoMaster.MasterContext; -import org.apache.tajo.master.rm.Worker; +import org.apache.tajo.master.rm.NodeStatus; import org.apache.tajo.ws.rs.JerseyResourceDelegate; import org.apache.tajo.ws.rs.JerseyResourceDelegateContext; import org.apache.tajo.ws.rs.JerseyResourceDelegateContextKey; @@ -99,11 +99,11 @@ public Response run(JerseyResourceDelegateContext context) { JerseyResourceDelegateContextKey.valueOf(JerseyResourceDelegateUtil.MasterContextKey, MasterContext.class); MasterContext masterContext = context.get(masterContextKey); - Map workerMap = masterContext.getResourceManager().getWorkers(); + Map workerMap = masterContext.getResourceManager().getNodes(); List workerList = new ArrayList(); - for (Worker worker: workerMap.values()) { - workerList.add(new WorkerResponse(worker)); + for (NodeStatus nodeStatus : workerMap.values()) { + workerList.add(new WorkerResponse(nodeStatus)); } Map> workerResponseMap = new HashMap>(); diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java index 854172c54d..d4c62e43fb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java +++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/responses/WorkerResponse.java @@ -18,8 +18,7 @@ package org.apache.tajo.ws.rs.responses; -import org.apache.tajo.master.rm.Worker; -import org.apache.tajo.master.rm.WorkerResource; +import org.apache.tajo.master.rm.NodeStatus; import com.google.gson.annotations.Expose; import org.apache.tajo.resource.NodeResource; @@ -41,13 +40,13 @@ public class WorkerResponse { @Expose private long lastHeartbeatTime; - public WorkerResponse(Worker worker) { - this(worker.getTotalResourceCapability(), worker.getAvailableResource(), - worker.getNumRunningTasks(), worker.getNumRunningQueryMaster()); + public WorkerResponse(NodeStatus nodeStatus) { + this(nodeStatus.getTotalResourceCapability(), nodeStatus.getAvailableResource(), + nodeStatus.getNumRunningTasks(), nodeStatus.getNumRunningQueryMaster()); - this.connectionInfo = new WorkerConnectionInfoResponse(worker.getConnectionInfo()); + this.connectionInfo = new WorkerConnectionInfoResponse(nodeStatus.getConnectionInfo()); - this.lastHeartbeatTime = worker.getLastHeartbeatTime(); + this.lastHeartbeatTime = nodeStatus.getLastHeartbeatTime(); } private WorkerResponse(NodeResource total, NodeResource available, int numRunningTasks, int numQueryMasterTasks) { diff --git a/tajo-core/src/main/resources/webapps/admin/cluster.jsp b/tajo-core/src/main/resources/webapps/admin/cluster.jsp index bcda1359ba..7aad8d4e29 100644 --- a/tajo-core/src/main/resources/webapps/admin/cluster.jsp +++ b/tajo-core/src/main/resources/webapps/admin/cluster.jsp @@ -23,13 +23,12 @@ <%@ page import="org.apache.tajo.master.cluster.WorkerConnectionInfo" %> <%@ page import="org.apache.tajo.service.ServiceTracker" %> <%@ page import="org.apache.tajo.service.TajoMasterInfo" %> -<%@ page import="org.apache.tajo.master.rm.Worker" %> -<%@ page import="org.apache.tajo.master.rm.WorkerState" %> +<%@ page import="org.apache.tajo.master.rm.NodeStatus" %> +<%@ page import="org.apache.tajo.master.rm.NodeState" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.TUtil" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> <%@ page import="java.util.*" %> -<%@ page import="org.apache.tajo.service.ServiceTracker" %> <%@ page import="java.net.InetSocketAddress" %> <% @@ -39,37 +38,37 @@ InetSocketAddress socketAddress = new InetSocketAddress(masterName[0], Integer.parseInt(masterName[1])); String masterLabel = socketAddress.getAddress().getHostName()+ ":" + socketAddress.getPort(); - Map workers = master.getContext().getResourceManager().getWorkers(); - List wokerKeys = new ArrayList(workers.keySet()); + Map nodes = master.getContext().getResourceManager().getNodes(); + List wokerKeys = new ArrayList(nodes.keySet()); Collections.sort(wokerKeys); int runningQueryMasterTasks = 0; - Set liveWorkers = new TreeSet(); - Set deadWorkers = new TreeSet(); - Set decommissionWorkers = new TreeSet(); + Set liveNodes = new TreeSet(); + Set deadNodes = new TreeSet(); + Set decommissionNodes = new TreeSet(); - Set liveQueryMasters = new TreeSet(); - Set deadQueryMasters = new TreeSet(); + Set liveQueryMasters = new TreeSet(); + Set deadQueryMasters = new TreeSet(); - for(Worker eachWorker: workers.values()) { - liveQueryMasters.add(eachWorker); - liveWorkers.add(eachWorker); - runningQueryMasterTasks += eachWorker.getNumRunningQueryMaster(); + for(NodeStatus eachNode: nodes.values()) { + liveQueryMasters.add(eachNode); + liveNodes.add(eachNode); + runningQueryMasterTasks += eachNode.getNumRunningQueryMaster(); } - for (Worker inactiveWorker : master.getContext().getResourceManager().getInactiveWorkers().values()) { - WorkerState state = inactiveWorker.getState(); + for (NodeStatus inactiveNode : master.getContext().getResourceManager().getInactiveNodes().values()) { + NodeState state = inactiveNode.getState(); - if (state == WorkerState.LOST) { - deadQueryMasters.add(inactiveWorker); - deadWorkers.add(inactiveWorker); - } else if (state == WorkerState.DECOMMISSIONED) { - decommissionWorkers.add(inactiveWorker); + if (state == NodeState.LOST) { + deadQueryMasters.add(inactiveNode); + deadNodes.add(inactiveNode); + } else if (state == NodeState.DECOMMISSIONED) { + decommissionNodes.add(inactiveNode); } } - String deadWorkersHtml = deadWorkers.isEmpty() ? "0": "" + deadWorkers.size() + ""; + String deadNodesHtml = deadNodes.isEmpty() ? "0": "" + deadNodes.size() + ""; String deadQueryMastersHtml = deadQueryMasters.isEmpty() ? "0": "" + deadQueryMasters.size() + ""; ServiceTracker haService = master.getContext().getHAService(); @@ -172,7 +171,7 @@ <% int no = 1; - for(Worker queryMaster: liveQueryMasters) { + for(NodeStatus queryMaster: liveQueryMasters) { WorkerConnectionInfo connectionInfo = queryMaster.getConnectionInfo(); String queryMasterHttp = "http://" + connectionInfo.getHost() + ":" + connectionInfo.getHttpInfoPort() + "/index.jsp"; @@ -204,7 +203,7 @@
    <% int no = 1; - for(Worker queryMaster: deadQueryMasters) { + for(NodeStatus queryMaster: deadQueryMasters) { %> @@ -220,32 +219,32 @@ %>
    -

    Worker

    -
    Live:<%=liveWorkers.size()%>, Dead: <%=deadWorkersHtml%>
    +

    Node

    +
    Live:<%=liveNodes.size()%>, Dead: <%=deadNodesHtml%>

    -

    Live Workers

    +

    Live Nodes

    <% - if(liveWorkers.isEmpty()) { - out.write("No Live Workers\n"); + if(liveNodes.isEmpty()) { + out.write("No Live Nodes\n"); } else { %>
    NoStartTimeFinishTimeRunTimeStatusFile Length# Messages
    <%=index%>
    NoQueryMaster
    <%=no++%>
    - + <% int no = 1; - for(Worker worker: liveWorkers) { - WorkerConnectionInfo connectionInfo = worker.getConnectionInfo(); - String workerHttp = "http://" + connectionInfo.getHost() + ":" + connectionInfo.getHttpInfoPort() + "/index.jsp"; + for(NodeStatus node: liveNodes) { + WorkerConnectionInfo connectionInfo = node.getConnectionInfo(); + String nodeHttp = "http://" + connectionInfo.getHost() + ":" + connectionInfo.getHttpInfoPort() + "/index.jsp"; %> - + - - - - - + + + + + <% } //end fo for @@ -258,24 +257,24 @@


    -

    Dead Workers

    +

    Dead Nodes

    <% - if(deadWorkers.isEmpty()) { + if(deadNodes.isEmpty()) { %> - No Dead Workers + No Dead Nodes <% } else { %>
    NoWorkerPullServer
    Port
    Running TasksAvailableTotalHeartbeatStatus
    NoNodePullServer
    Port
    Running TasksAvailableTotalHeartbeatStatus
    <%=no++%><%=connectionInfo.getHostAndPeerRpcPort()%><%=connectionInfo.getHostAndPeerRpcPort()%> <%=connectionInfo.getPullServerPort()%><%=worker.getNumRunningTasks()%><%=worker.getAvailableResource()%><%=worker.getTotalResourceCapability()%><%=JSPUtil.getElapsedTime(worker.getLastHeartbeatTime(), System.currentTimeMillis())%><%=worker.getState()%><%=node.getNumRunningTasks()%><%=node.getAvailableResource()%><%=node.getTotalResourceCapability()%><%=JSPUtil.getElapsedTime(node.getLastHeartbeatTime(), System.currentTimeMillis())%><%=node.getState()%>
    - + <% int no = 1; - for(Worker worker: deadWorkers) { + for(NodeStatus node: deadNodes) { %> - + <% } //end fo for diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp index e8862212d2..96facc541a 100644 --- a/tajo-core/src/main/resources/webapps/admin/index.jsp +++ b/tajo-core/src/main/resources/webapps/admin/index.jsp @@ -24,10 +24,8 @@ <%@ page import="org.apache.tajo.master.TajoMaster" %> <%@ page import="org.apache.tajo.service.ServiceTracker" %> <%@ page import="org.apache.tajo.service.TajoMasterInfo" %> -<%@ page import="org.apache.tajo.master.rm.Worker" %> -<%@ page import="org.apache.tajo.master.rm.WorkerState" %> -<%@ page import="org.apache.tajo.service.ServiceTracker" %> -<%@ page import="org.apache.tajo.service.TajoMasterInfo" %> +<%@ page import="org.apache.tajo.master.rm.NodeStatus" %> +<%@ page import="org.apache.tajo.master.rm.NodeState" %> <%@ page import="org.apache.tajo.storage.TablespaceManager" %> <%@ page import="org.apache.tajo.storage.Tablespace" %> <%@ page import="org.apache.tajo.util.NetUtils" %> @@ -36,7 +34,6 @@ <%@ page import="java.util.List" %> <%@ page import="java.net.InetSocketAddress" %> <%@ page import="java.util.Date" %> -<%@ page import="java.util.List" %> <%@ page import="java.util.Map" %> <% @@ -46,8 +43,8 @@ InetSocketAddress socketAddress = new InetSocketAddress(masterName[0], Integer.parseInt(masterName[1])); String masterLabel = socketAddress.getAddress().getHostName()+ ":" + socketAddress.getPort(); - Map workers = master.getContext().getResourceManager().getWorkers(); - Map inactiveWorkers = master.getContext().getResourceManager().getInactiveWorkers(); + Map workers = master.getContext().getResourceManager().getNodes(); + Map inactiveWorkers = master.getContext().getResourceManager().getInactiveNodes(); int numWorkers = 0; int numLiveWorkers = 0; @@ -60,7 +57,7 @@ int runningQueryMasterTask = 0; - for(Worker eachWorker: workers.values()) { + for(NodeStatus eachWorker: workers.values()) { numQueryMasters++; numLiveQueryMasters++; runningQueryMasterTask += eachWorker.getNumRunningQueryMaster(); @@ -68,13 +65,13 @@ numLiveWorkers++; } - for (Worker eachWorker : inactiveWorkers.values()) { - if (eachWorker.getState() == WorkerState.LOST) { + for (NodeStatus eachWorker : inactiveWorkers.values()) { + if (eachWorker.getState() == NodeState.LOST) { numQueryMasters++; numDeadQueryMasters++; numWorkers++; numDeadWorkers++; - } else if(eachWorker.getState() == WorkerState.DECOMMISSIONED) { + } else if(eachWorker.getState() == NodeState.DECOMMISSIONED) { numDecommissionWorkers++; } } diff --git a/tajo-core/src/main/resources/webapps/admin/query.jsp b/tajo-core/src/main/resources/webapps/admin/query.jsp index bc1a04b41a..0701e34ed2 100644 --- a/tajo-core/src/main/resources/webapps/admin/query.jsp +++ b/tajo-core/src/main/resources/webapps/admin/query.jsp @@ -21,7 +21,7 @@ <%@ page import="org.apache.tajo.master.TajoMaster" %> <%@ page import="org.apache.tajo.master.QueryInProgress" %> -<%@ page import="org.apache.tajo.master.rm.Worker" %> +<%@ page import="org.apache.tajo.master.rm.NodeStatus" %> <%@ page import="org.apache.tajo.util.JSPUtil" %> <%@ page import="org.apache.tajo.util.StringUtils" %> <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %> @@ -70,15 +70,15 @@ SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - Map workers = master.getContext().getResourceManager().getWorkers(); + Map workers = master.getContext().getResourceManager().getNodes(); Map portMap = new HashMap(); Collection queryMasters = master.getContext().getResourceManager().getQueryMasters(); if (queryMasters == null || queryMasters.isEmpty()) { - queryMasters = master.getContext().getResourceManager().getWorkers().keySet(); + queryMasters = master.getContext().getResourceManager().getNodes().keySet(); } for(int eachQueryMasterKey: queryMasters) { - Worker queryMaster = workers.get(eachQueryMasterKey); + NodeStatus queryMaster = workers.get(eachQueryMasterKey); if(queryMaster != null) { portMap.put(queryMaster.getConnectionInfo().getHost(), queryMaster.getConnectionInfo().getHttpInfoPort()); } diff --git a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp index 27b189b5e1..7c73061fc3 100644 --- a/tajo-core/src/main/resources/webapps/admin/querytasks.jsp +++ b/tajo-core/src/main/resources/webapps/admin/querytasks.jsp @@ -26,7 +26,7 @@ <%@ page import="org.apache.tajo.util.history.HistoryReader" %> <%@ page import="org.apache.tajo.util.history.QueryHistory" %> <%@ page import="org.apache.tajo.util.history.StageHistory" %> -<%@ page import="org.apache.tajo.master.rm.Worker" %> +<%@ page import="org.apache.tajo.master.rm.NodeStatus" %> <%@ page import="java.util.*" %> <%@ page import="org.apache.tajo.util.history.TaskHistory" %> <%@ page import="org.apache.tajo.util.*" %> @@ -74,11 +74,11 @@ status = "ALL"; } - Collection allWorkers = master.getContext().getResourceManager().getWorkers().values(); + Collection allWorkers = master.getContext().getResourceManager().getNodes().values(); - Map workerMap = new HashMap(); + Map workerMap = new HashMap(); if(allWorkers != null) { - for(Worker eachWorker: allWorkers) { + for(NodeStatus eachWorker: allWorkers) { workerMap.put(eachWorker.getConnectionInfo().getHostAndPeerRpcPort(), eachWorker); } } @@ -218,7 +218,7 @@ } String taskHost = eachTask.getHostAndPort() == null ? "-" : eachTask.getHostAndPort(); if (eachTask.getHostAndPort() != null) { - Worker worker = workerMap.get(eachTask.getHostAndPort()); + NodeStatus worker = workerMap.get(eachTask.getHostAndPort()); if (worker != null) { String[] hostTokens = eachTask.getHostAndPort().split(":"); taskHost = " allWorkers = master.getContext().getResourceManager().getNodes().values(); - Map workerMap = new HashMap(); + Map nodeMap = new HashMap(); if(allWorkers != null) { for(NodeStatus eachWorker: allWorkers) { - workerMap.put(eachWorker.getConnectionInfo().getHostAndPeerRpcPort(), eachWorker); + nodeMap.put(eachWorker.getConnectionInfo().getHostAndPeerRpcPort(), eachWorker); } } @@ -218,10 +218,10 @@ } String taskHost = eachTask.getHostAndPort() == null ? "-" : eachTask.getHostAndPort(); if (eachTask.getHostAndPort() != null) { - NodeStatus worker = workerMap.get(eachTask.getHostAndPort()); - if (worker != null) { + NodeStatus nodeStatus = nodeMap.get(eachTask.getHostAndPort()); + if (nodeStatus != null) { String[] hostTokens = eachTask.getHostAndPort().split(":"); - taskHost = "" + eachTask.getHostAndPort() + ""; } From 42c23c7bbf69803d5e32c302d135c9fdb550aa68 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 17 Jul 2015 18:25:24 +0900 Subject: [PATCH 74/80] fix jdk6 build failure --- .../apache/tajo/master/scheduler/SchedulingAlgorithms.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java index 365897e150..efe45619e3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java @@ -18,6 +18,8 @@ package org.apache.tajo.master.scheduler; +import org.apache.tajo.util.NumberUtil; + import java.util.Comparator; /** @@ -32,9 +34,9 @@ public class SchedulingAlgorithms { public static class FifoComparator implements Comparator { @Override public int compare(QuerySchedulingInfo q1, QuerySchedulingInfo q2) { - int res = Integer.compare(q1.getPriority(), q2.getPriority()); + int res = NumberUtil.compare(q1.getPriority(), q2.getPriority()); if (res == 0) { - res = Long.compare(q1.getStartTime(), q2.getStartTime()); + res = NumberUtil.compare(q1.getStartTime(), q2.getStartTime()); } if (res == 0) { // In the rare case where jobs were submitted at the exact same time, From 0e3bb654fd6d5dc956664a36cfcfcb03ddc3dc10 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sat, 18 Jul 2015 17:48:12 +0900 Subject: [PATCH 75/80] add missing tail task management --- .../java/org/apache/tajo/conf/TajoConf.java | 2 +- .../master/scheduler/SimpleScheduler.java | 2 +- .../querymaster/DefaultTaskScheduler.java | 69 +++++++++++++------ .../org/apache/tajo/querymaster/Query.java | 2 +- .../java/org/apache/tajo/worker/TaskImpl.java | 17 ++--- 5 files changed, 59 insertions(+), 33 deletions(-) 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 7939d30f19..0436116919 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 @@ -194,7 +194,7 @@ public static enum ConfVars implements ConfigKey { RESOURCE_SCHEDULER_CLASS("tajo.resource.scheduler", "org.apache.tajo.master.scheduler.SimpleScheduler", Validators.groups(Validators.notNull(), Validators.clazz())), - QUERYMASTER_TASK_SCHEDULER_DELAY("tajo.qm.task.task-scheduler.delay", 50), // 50 ms + QUERYMASTER_TASK_SCHEDULER_DELAY("tajo.qm.task-scheduler.delay", 50), // 50 ms // Catalog CATALOG_ADDRESS("tajo.catalog.client-rpc.address", "localhost:26005", Validators.networkAddr()), diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java index 6432f9875d..e41ac95980 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleScheduler.java @@ -226,7 +226,7 @@ private List reserveClusterResource(List worke NodeStatus nodeStatus = getRMContext().getNodes().get(workerId); if (nodeStatus == null) { iter.remove(); - LOG.warn("Can't found the nodeStatus :" + workerId); + LOG.warn("Can't find the node. id :" + workerId); continue; } else { if (NodeResources.fitsIn(capacity, nodeStatus.getAvailableResource())) { diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java index 85a65bbd6c..32e421932a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java @@ -63,8 +63,11 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler { private static final Log LOG = LogFactory.getLog(DefaultTaskScheduler.class); + private static final String REQUEST_MAX_NUM = "tajo.qm.task-scheduler.request.max-num"; + private final TaskSchedulerContext context; private Stage stage; + private TajoConf tajoConf; private Thread schedulingThread; private volatile boolean isStopped; @@ -76,6 +79,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler { private int scheduledObjectNum = 0; private boolean isLeaf; private int schedulerDelay; + private int maximumRequestContainer; //candidate workers for locality of high priority private Set candidateWorkers = Sets.newHashSet(); @@ -88,17 +92,17 @@ public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) { @Override public void init(Configuration conf) { - TajoConf tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); + tajoConf = TUtil.checkTypeAndGet(conf, TajoConf.class); scheduledRequests = new ScheduledRequests(); minTaskMemory = tajoConf.getIntVar(TajoConf.ConfVars.TASK_RESOURCE_MINIMUM_MEMORY); schedulerDelay= tajoConf.getIntVar(TajoConf.ConfVars.QUERYMASTER_TASK_SCHEDULER_DELAY); - super.init(conf); } @Override public void start() { LOG.info("Start TaskScheduler"); + maximumRequestContainer = tajoConf.getInt(REQUEST_MAX_NUM, stage.getContext().getWorkerMap().size() * 2); isLeaf = stage.getMasterPlan().isLeaf(stage.getBlock()); if (isLeaf) { @@ -119,6 +123,13 @@ public void run() { try { schedule(); + } catch (InterruptedException e) { + if (isStopped) { + break; + } else { + LOG.fatal(e.getMessage(), e); + stage.abort(StageState.ERROR); + } } catch (Throwable e) { LOG.fatal(e.getMessage(), e); stage.abort(StageState.ERROR); @@ -139,7 +150,7 @@ public void stop() { if (schedulingThread != null) { synchronized (schedulingThread) { - schedulingThread.notifyAll(); + schedulingThread.interrupt(); } } candidateWorkers.clear(); @@ -266,7 +277,9 @@ private Set getWorkerIds(Collection hosts){ protected LinkedList createTaskRequest() throws Exception { LinkedList taskRequestEvents = new LinkedList(); - int requestContainerNum = remainingScheduledObjectNum(); + //If scheduled tasks is long-term task, cluster resource can be the worst load balance. + //This part is to throttle the maximum required container per request + int requestContainerNum = Math.min(remainingScheduledObjectNum(), maximumRequestContainer); if (LOG.isDebugEnabled()) { LOG.debug("Try to schedule task resources: " + requestContainerNum); } @@ -554,15 +567,6 @@ public int getLowestVolumeId(){ } } - public boolean isRemote(TaskAttemptId taskAttemptId){ - Integer volumeId = lastAssignedVolumeId.get(taskAttemptId); - if(volumeId == null || volumeId > REMOTE){ - return false; - } else { - return true; - } - } - public int getRemoteConcurrency(){ return getVolumeConcurrency(REMOTE); } @@ -578,17 +582,12 @@ public int getRemainingLocalTaskSize(){ } public String getHost() { - return host; } public String getRack() { return rack; } - - public int getAssignedVolumeId(TaskAttemptId attemptId) { - return lastAssignedVolumeId.get(attemptId); - } } public void cancel(TaskAttempt taskAttempt) { @@ -788,16 +787,18 @@ public void assignToLeafTasks(LinkedList taskRequests) { String host = connectionInfo.getHost(); // if there are no worker matched to the hostname a task request - if(!leafTaskHostMapping.containsKey(host)){ + if (!leafTaskHostMapping.containsKey(host) && !taskRequests.isEmpty()) { String normalizedHost = NetUtils.normalizeHost(host); - if(!leafTaskHostMapping.containsKey(normalizedHost) && !taskRequests.isEmpty()){ + if (!leafTaskHostMapping.containsKey(normalizedHost)) { // this case means one of either cases: // * there are no blocks which reside in this node. // * all blocks which reside in this node are consumed, and this task runner requests a remote task. // In this case, we transfer the task request to the remote task request list, and skip the followings. remoteTaskRequests.add(taskRequest); continue; + } else { + host = normalizedHost; } } @@ -812,10 +813,38 @@ public void assignToLeafTasks(LinkedList taskRequests) { TaskAttemptId attemptId = allocateLocalTask(host); if (attemptId == null) { // if a local task cannot be found + HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host); + + if(!taskRequests.isEmpty()) { //if other requests remains, move to remote list for better locality + remoteTaskRequests.add(taskRequest); + candidateWorkers.remove(connectionInfo.getId()); + continue; + + } else { + if(hostVolumeMapping != null) { + int nodes = context.getMasterContext().getWorkerMap().size(); + //this part is to control the assignment of tail and remote task balancing per node + int tailLimit = 1; + if (remainingScheduledObjectNum() > 0) { + tailLimit = Math.max(remainingScheduledObjectNum() / nodes, 1); + } + + if (hostVolumeMapping.getRemoteConcurrency() >= tailLimit) { //remote task throttling per node + continue; + } else { + // assign to remote volume + hostVolumeMapping.increaseConcurrency(HostVolumeMapping.REMOTE); + } + } + } + ////////////////////////////////////////////////////////////////////// // rack-local allocation ////////////////////////////////////////////////////////////////////// attemptId = allocateRackTask(host); + if (attemptId != null && hostVolumeMapping != null) { + hostVolumeMapping.lastAssignedVolumeId.put(attemptId, HostVolumeMapping.REMOTE); + } ////////////////////////////////////////////////////////////////////// // random node allocation diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java index ec57aa8b89..6fc4ea9d22 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java @@ -714,7 +714,7 @@ public void transition(Query query, QueryEvent event) { } //wait for stages is completed - if (query.completedStagesCount == query.stages.size()) { + if (query.completedStagesCount >= query.stages.size()) { // if a query is completed due to finished, kill, failure, or error query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState())); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java index 4c1090f75c..d77c583b0e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -467,6 +467,7 @@ public void run() throws Exception { @Override public void cleanup() { + // history store in memory while running stage TaskHistory taskHistory = createTaskHistory(); executionBlockContext.addTaskHistory(getId().getTaskId(), taskHistory); executionBlockContext.getTasks().remove(getId()); @@ -510,16 +511,12 @@ public TaskHistory createTaskHistory() { int i = 0; FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); for (Fetcher fetcher : fetcherRunners) { - // TODO store the fetcher histories - if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { - builder.setStartTime(fetcher.getStartTime()); - builder.setFinishTime(fetcher.getFinishTime()); - builder.setFileLength(fetcher.getFileLen()); - builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); - builder.setState(fetcher.getState()); - - taskHistory.addFetcherHistory(builder.build()); - } + builder.setStartTime(fetcher.getStartTime()); + builder.setFinishTime(fetcher.getFinishTime()); + builder.setFileLength(fetcher.getFileLen()); + builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); + builder.setState(fetcher.getState()); + taskHistory.addFetcherHistory(builder.build()); if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; } taskHistory.setFinishedFetchCount(i); From 429d941c888a2831d1891166c23584f99a37bccd Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Sat, 18 Jul 2015 18:38:29 +0900 Subject: [PATCH 76/80] remove unused methods --- .travis.yml | 1 - .../main/java/org/apache/tajo/util/TUtil.java | 1 - .../apache/tajo/engine/query/TaskRequest.java | 4 -- .../tajo/engine/query/TaskRequestImpl.java | 39 ------------------- .../event/TaskAttemptToSchedulerEvent.java | 6 --- .../org/apache/tajo/master/rm/NodeStatus.java | 6 ++- 6 files changed, 4 insertions(+), 53 deletions(-) diff --git a/.travis.yml b/.travis.yml index c374445a9b..8c3cce6d3b 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,5 +1,4 @@ # 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 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 0a77e5ca36..66e8acca90 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 @@ -19,7 +19,6 @@ package org.apache.tajo.util; import com.google.common.base.Objects; -import org.apache.tajo.conf.TajoConf; import java.lang.reflect.Array; import java.util.*; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java index 98ccdb3c7e..48d47807b8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequest.java @@ -35,13 +35,9 @@ public interface TaskRequest extends ProtoObject { - String getQueryMasterHostAndPort(); TaskAttemptId getId(); List getFragments(); - String getOutputTableId(); - boolean isClusteredOutput(); PlanProto.LogicalNodeTree getPlan(); - boolean isInterQuery(); void setInterQuery(); void addFetch(String name, FetchImpl fetch); List getFetches(); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java index 6355046ff4..f97d00507f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/TaskRequestImpl.java @@ -98,19 +98,6 @@ public TaskRequestProto getProto() { return proto; } - @Override - public String getQueryMasterHostAndPort() { - TaskRequestProtoOrBuilder p = viaProto ? proto : builder; - if (queryMasterHostAndPort != null) { - return this.queryMasterHostAndPort; - } - if (!p.hasQueryMasterHostAndPort()) { - return null; - } - this.queryMasterHostAndPort = p.getQueryMasterHostAndPort(); - return this.queryMasterHostAndPort; - } - @Override public TaskAttemptId getId() { TaskRequestProtoOrBuilder p = viaProto ? proto : builder; @@ -139,32 +126,6 @@ public List getFragments() { return this.fragments; } - @Override - public String getOutputTableId() { - TaskRequestProtoOrBuilder p = viaProto ? proto : builder; - if (outputTable != null) { - return this.outputTable; - } - if (!p.hasOutputTable()) { - return null; - } - this.outputTable = p.getOutputTable(); - return this.outputTable; - } - - @Override - public boolean isClusteredOutput() { - TaskRequestProtoOrBuilder p = viaProto ? proto : builder; - if (isUpdated) { - return this.clusteredOutput; - } - if (!p.hasClusteredOutput()) { - return false; - } - this.clusteredOutput = p.getClusteredOutput(); - this.isUpdated = true; - return this.clusteredOutput; - } @Override public PlanProto.LogicalNodeTree getPlan() { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java index 1eee40d052..6799ce1c75 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java @@ -50,12 +50,6 @@ public TaskAttemptScheduleContext() { } - public TaskAttemptScheduleContext(String host, - RpcCallback callback) { - this.host = host; - this.callback = callback; - } - public String getHost() { return host; } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatus.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatus.java index fc1834b9c2..63e4d523ed 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatus.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/NodeStatus.java @@ -45,8 +45,10 @@ public class NodeStatus implements EventHandler, Comparable Date: Sat, 18 Jul 2015 19:09:15 +0900 Subject: [PATCH 77/80] fix broken unit test --- .../NonForwardQueryResultSystemScanner.java | 2 +- ...estNonForwardQueryResultSystemScanner.java | 42 +------------------ 2 files changed, 2 insertions(+), 42 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java index 58837a0623..0ace53a36f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java @@ -485,7 +485,7 @@ private Tuple getWorkerTuple(Schema outSchema, NodeStatus aNodeStatus) { aTuple.put(fieldId, DatumFactory.createNullDatum()); } } else if ("type".equalsIgnoreCase(column.getSimpleName())) { - aTuple.put(fieldId, DatumFactory.createText("NodeStatus")); + aTuple.put(fieldId, DatumFactory.createText("Worker")); } else if ("status".equalsIgnoreCase(column.getSimpleName())) { aTuple.put(fieldId, DatumFactory.createText(aNodeStatus.getState().toString())); } else if ("RUNNING".equalsIgnoreCase(aNodeStatus.getState().toString())) { diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java index e8d59d0d5e..207f64db67 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java @@ -18,49 +18,9 @@ package org.apache.tajo.master; -import static org.junit.Assert.*; -import static org.hamcrest.CoreMatchers.*; - -import java.io.File; -import java.sql.ResultSet; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.UUID; - -import org.apache.tajo.*; -import org.apache.tajo.algebra.Expr; -import org.apache.tajo.benchmark.TPCH; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.client.ResultSetUtil; -import org.apache.tajo.client.TajoClient; -import org.apache.tajo.common.TajoDataTypes.Type; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.master.TajoMaster.MasterContext; -import org.apache.tajo.master.exec.NonForwardQueryResultScanner; -import org.apache.tajo.master.exec.NonForwardQueryResultSystemScanner; -import org.apache.tajo.plan.LogicalOptimizer; -import org.apache.tajo.plan.LogicalPlan; -import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.logical.LimitNode; -import org.apache.tajo.plan.logical.NodeType; -import org.apache.tajo.storage.RowStoreUtil; -import org.apache.tajo.storage.StorageConstants; -import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder; -import org.apache.tajo.storage.Tuple; -import org.apache.tajo.util.KeyValueSet; -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeDiagnosingMatcher; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.apache.tajo.QueryTestCaseBase; import org.junit.Test; -import com.google.protobuf.ByteString; - public class TestNonForwardQueryResultSystemScanner extends QueryTestCaseBase { @Test public void testGetNextRowsForAggregateFunction() throws Exception { From 6ca6a9ba83b34d24aa72f1659360a2d29d125ea3 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 20 Jul 2015 12:12:18 +0900 Subject: [PATCH 78/80] fix tajo-web compilation error on jdk8 --- tajo-core/pom.xml | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index 19c0e2ca4b..1b0d44008e 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -390,11 +390,6 @@ dspace-geoip 1.2.3 - - org.eclipse.jdt - core - 3.1.1 - io.netty netty-codec-http @@ -408,6 +403,12 @@ org.mortbay.jetty jsp-2.1 6.1.14 + + + core + org.eclipse.jdt + + com.codahale.metrics From a11b3b3b11f459227c3a938ee1fda468b9e57a3c Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 20 Jul 2015 13:46:45 +0900 Subject: [PATCH 79/80] increse max user process for travis --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 8c3cce6d3b..671a31f0e3 100644 --- a/.travis.yml +++ b/.travis.yml @@ -33,7 +33,7 @@ notifications: - issues@tajo.apache.org irc: "chat.freenode.net#tajo" -before_install: ulimit -t 514029 -n 3000 +before_install: ulimit -t 514029 -u 2048 -n 3000 install: ./dev-support/travis-install-dependencies.sh script: From 1508fac8258931eac967110438ad37add887317c Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 20 Jul 2015 14:21:32 +0900 Subject: [PATCH 80/80] fix wrong unit test --- .../tajo/master/scheduler/TestSimpleScheduler.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java index 01b199850f..36fd9393b9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java +++ b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestSimpleScheduler.java @@ -242,6 +242,7 @@ public void testReserveResourceWithWorkerPriority() throws InterruptedException, class MySimpleScheduler extends SimpleScheduler { Semaphore barrier; Map queryInfoMap = Maps.newHashMap(); + Map qmAllocationMap = Maps.newHashMap(); public MySimpleScheduler(TajoRMContext rmContext, Semaphore barrier) { super(null, rmContext); @@ -265,13 +266,12 @@ public QueryContext getQueryContext() { } @Override - protected boolean startQuery(QueryId queryId, final AllocationResourceProto allocation) { + protected boolean startQuery(final QueryId queryId, final AllocationResourceProto allocation) { executorService.schedule(new Runnable() { @Override public void run() { barrier.release(); - NodeResources.addTo(rmContext.getNodes().get(allocation.getWorkerId()).getAvailableResource(), - new NodeResource(allocation.getResource())); + qmAllocationMap.put(queryId, allocation); rmContext.getDispatcher().getEventHandler().handle(new SchedulerEvent(SchedulerEventType.RESOURCE_UPDATE)); } }, testDelay, TimeUnit.MILLISECONDS); @@ -292,6 +292,9 @@ protected QueryInfo getQueryInfo(QueryId queryId) { @Override public void stopQuery(QueryId queryId) { queryInfoMap.remove(queryId); + AllocationResourceProto allocationResourceProto = qmAllocationMap.remove(queryId); + NodeResources.addTo(rmContext.getNodes().get(allocationResourceProto.getWorkerId()).getAvailableResource(), + new NodeResource(allocationResourceProto.getResource())); super.stopQuery(queryId); } }
    NoWorker
    NoNode
    <%=no++%><%=worker.getConnectionInfo().getHostAndPeerRpcPort()%><%=node.getConnectionInfo().getHostAndPeerRpcPort()%>