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

[5/5] git commit: TAJO-255: Cleanup exceptions of engine. (hyunsik)

TAJO-255: Cleanup exceptions of engine. (hyunsik)


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

Branch: refs/heads/master
Commit: 3f45f0cdde6c15a820a19f5b5c5a52f7f0322ed8
Parents: 660606b
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Oct 16 18:28:01 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Oct 16 18:28:01 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |    2 +
 .../main/java/org/apache/tajo/DataChannel.java  |  198 --
 .../org/apache/tajo/TaskAttemptContext.java     |  231 --
 .../org/apache/tajo/client/ResultSetImpl.java   | 2173 +++++++++++++++++
 .../tajo/client/ResultSetMetaDataImpl.java      |  259 +++
 .../java/org/apache/tajo/client/TajoClient.java |    1 -
 .../exception/AmbiguousFieldException.java      |   30 +
 .../engine/exception/EmptyClusterException.java |   34 -
 .../engine/exception/InvalidQueryException.java |   35 +
 .../engine/exception/NoSuchColumnException.java |   25 +
 .../exception/NoSuchQueryIdException.java       |   43 -
 .../exception/RangeOverflowException.java       |   28 +
 .../exception/UndefinedFunctionException.java   |   34 +
 .../exception/UnfinishedTaskException.java      |   35 -
 .../exception/UnknownWorkerException.java       |   44 -
 .../tajo/engine/exception/VerifyException.java  |   27 +
 .../tajo/engine/parser/HiveConverter.java       |    2 -
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |    2 -
 .../tajo/engine/parser/SQLErrorListener.java    |    1 -
 .../tajo/engine/parser/SQLParseError.java       |  107 +
 .../tajo/engine/parser/SQLSyntaxError.java      |   51 +
 .../apache/tajo/engine/planner/LogicalPlan.java |    2 +
 .../tajo/engine/planner/LogicalPlanner.java     |    5 +-
 .../engine/planner/NoSuchColumnException.java   |   25 -
 .../tajo/engine/planner/PhysicalPlanner.java    |    2 +-
 .../engine/planner/PhysicalPlannerImpl.java     |    4 +-
 .../apache/tajo/engine/planner/PlannerUtil.java |    2 +-
 .../engine/planner/RangeOverflowException.java  |   28 -
 .../engine/planner/UniformRangePartition.java   |    1 +
 .../tajo/engine/planner/VerifyException.java    |   25 -
 .../tajo/engine/planner/global/DataChannel.java |  198 ++
 .../engine/planner/global/ExecutionBlock.java   |  160 ++
 .../planner/global/ExecutionBlockCursor.java    |   71 +
 .../engine/planner/global/GlobalPlanner.java    |  588 +++++
 .../tajo/engine/planner/global/MasterPlan.java  |    5 +-
 .../planner/physical/AggregationExec.java       |    2 +-
 .../engine/planner/physical/BNLJoinExec.java    |    2 +-
 .../planner/physical/BSTIndexScanExec.java      |    2 +-
 .../planner/physical/BinaryPhysicalExec.java    |    2 +-
 .../engine/planner/physical/EvalExprExec.java   |    2 +-
 .../planner/physical/ExternalSortExec.java      |    2 +-
 .../planner/physical/HashAggregateExec.java     |    2 +-
 .../planner/physical/HashFullOuterJoinExec.java |    2 +-
 .../engine/planner/physical/HashJoinExec.java   |    2 +-
 .../planner/physical/HashLeftAntiJoinExec.java  |    2 +-
 .../planner/physical/HashLeftOuterJoinExec.java |    2 +-
 .../planner/physical/HashLeftSemiJoinExec.java  |    2 +-
 .../planner/physical/IndexedStoreExec.java      |    2 +-
 .../tajo/engine/planner/physical/LimitExec.java |    2 +-
 .../engine/planner/physical/MemSortExec.java    |    2 +-
 .../physical/MergeFullOuterJoinExec.java        |    2 +-
 .../engine/planner/physical/MergeJoinExec.java  |    2 +-
 .../engine/planner/physical/NLJoinExec.java     |    2 +-
 .../planner/physical/NLLeftOuterJoinExec.java   |    2 +-
 .../planner/physical/PartitionedStoreExec.java  |    2 +-
 .../engine/planner/physical/PhysicalExec.java   |    2 +-
 .../engine/planner/physical/ProjectionExec.java |    2 +-
 .../physical/RightOuterMergeJoinExec.java       |    2 +-
 .../engine/planner/physical/SelectionExec.java  |    2 +-
 .../engine/planner/physical/SeqScanExec.java    |    2 +-
 .../planner/physical/SortAggregateExec.java     |    2 +-
 .../tajo/engine/planner/physical/SortExec.java  |    2 +-
 .../engine/planner/physical/StoreTableExec.java |    2 +-
 .../engine/planner/physical/TunnelExec.java     |    2 +-
 .../planner/physical/UnaryPhysicalExec.java     |    2 +-
 .../tajo/engine/planner/physical/UnionExec.java |    4 +-
 .../planner/rewrite/FilterPushDownRule.java     |    2 +-
 .../apache/tajo/engine/query/QueryContext.java  |  179 ++
 .../tajo/engine/query/QueryUnitRequest.java     |   50 +
 .../tajo/engine/query/QueryUnitRequestImpl.java |    4 +-
 .../apache/tajo/engine/query/ResultSetImpl.java | 2174 ------------------
 .../engine/query/ResultSetMetaDataImpl.java     |  259 ---
 .../exception/AmbiguousFieldException.java      |   30 -
 .../query/exception/InvalidQueryException.java  |   35 -
 .../exception/NotSupportQueryException.java     |   34 -
 .../engine/query/exception/SQLParseError.java   |  107 -
 .../engine/query/exception/SQLSyntaxError.java  |   49 -
 .../exception/UndefinedFunctionException.java   |   34 -
 .../ipc/protocolrecords/QueryUnitRequest.java   |   51 -
 .../org/apache/tajo/master/ExecutionBlock.java  |  160 --
 .../tajo/master/ExecutionBlockCursor.java       |   74 -
 .../org/apache/tajo/master/GlobalEngine.java    |    8 +-
 .../org/apache/tajo/master/GlobalPlanner.java   |  590 -----
 .../org/apache/tajo/master/QueryContext.java    |  179 --
 .../apache/tajo/master/TaskSchedulerImpl.java   |    2 +-
 .../tajo/master/event/QueryStartEvent.java      |    2 +-
 .../apache/tajo/master/querymaster/Query.java   |    8 +-
 .../master/querymaster/QueryInProgress.java     |    2 +-
 .../master/querymaster/QueryJobManager.java     |    2 +-
 .../tajo/master/querymaster/QueryMaster.java    |   25 +-
 .../master/querymaster/QueryMasterTask.java     |   13 +-
 .../tajo/master/querymaster/Repartitioner.java  |    4 +-
 .../tajo/master/querymaster/SubQuery.java       |    4 +-
 .../tajo/scheduler/event/ScheduleTaskEvent.java |   34 -
 .../tajo/scheduler/event/SchedulerEvent.java    |   28 -
 .../scheduler/event/SchedulerEventType.java     |   27 -
 .../org/apache/tajo/worker/TajoQueryEngine.java |    3 -
 .../java/org/apache/tajo/worker/TajoWorker.java |    1 +
 .../tajo/worker/TajoWorkerClientService.java    |   10 +-
 .../tajo/worker/TajoWorkerManagerService.java   |    2 +-
 .../main/java/org/apache/tajo/worker/Task.java  |   13 +-
 .../apache/tajo/worker/TaskAttemptContext.java  |  233 ++
 .../main/resources/webapps/worker/queryplan.jsp |    4 +-
 .../org/apache/tajo/TajoTestingCluster.java     |   15 +-
 .../org/apache/tajo/client/TestTajoClient.java  |   12 +-
 .../tajo/engine/parser/TestHiveConverter.java   |    2 -
 .../engine/planner/global/TestMasterPlan.java   |    1 -
 .../planner/physical/TestBNLJoinExec.java       |    2 +-
 .../planner/physical/TestBSTIndexExec.java      |    2 +-
 .../planner/physical/TestExternalSortExec.java  |    2 +-
 .../physical/TestFullOuterHashJoinExec.java     |    2 +-
 .../physical/TestFullOuterMergeJoinExec.java    |    2 +-
 .../planner/physical/TestHashAntiJoinExec.java  |    2 +-
 .../planner/physical/TestHashJoinExec.java      |    2 +-
 .../planner/physical/TestHashSemiJoinExec.java  |    2 +-
 .../physical/TestLeftOuterHashJoinExec.java     |    2 +-
 .../physical/TestLeftOuterNLJoinExec.java       |    2 +-
 .../planner/physical/TestMergeJoinExec.java     |    2 +-
 .../engine/planner/physical/TestNLJoinExec.java |    2 +-
 .../planner/physical/TestPhysicalPlanner.java   |    2 +
 .../physical/TestRightOuterHashJoinExec.java    |    2 +-
 .../physical/TestRightOuterMergeJoinExec.java   |    4 +-
 .../engine/planner/physical/TestSortExec.java   |    2 +-
 .../tajo/engine/query/TestInsertQuery.java      |   48 +-
 .../tajo/engine/query/TestResultSetImpl.java    |    1 +
 .../tajo/master/TestExecutionBlockCursor.java   |    5 +-
 .../apache/tajo/master/TestGlobalPlanner.java   |    2 +
 .../tajo/worker/TestRangeRetrieverHandler.java  |    1 -
 128 files changed, 4418 insertions(+), 4681 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 50625bd..074416f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -52,6 +52,8 @@ Release 0.2.0 - unreleased
 
   IMPROVEMENTS
 
+    TAJO-255: Cleanup exceptions of engine. (hyunsik)
+
     TAJO-253: Clean up tajo-default.xml in test resource. (hyunsik)
 
     TAJO-239: Improving web UI. (Keuntae Park via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java
deleted file mode 100644
index e00d534..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo;
-
-import com.google.common.base.Preconditions;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.master.ExecutionBlock;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import static org.apache.tajo.ipc.TajoWorkerProtocol.*;
-
-public class DataChannel {
-  private ExecutionBlockId srcId;
-  private ExecutionBlockId targetId;
-  private TransmitType transmitType = TransmitType.PULL_TRANSMIT;
-  private PartitionType partitionType;
-  private Integer partitionNum = 1;
-  private Column[] key;
-
-  private Schema schema;
-
-  private StoreType storeType = StoreType.CSV;
-
-  public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId) {
-    this.srcId = srcId;
-    this.targetId = targetId;
-  }
-
-  public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId, PartitionType partitionType) {
-    this(srcId, targetId);
-    this.partitionType = partitionType;
-  }
-
-  public DataChannel(ExecutionBlock src, ExecutionBlock target, PartitionType partitionType, int partNum) {
-    this(src.getId(), target.getId(), partitionType, partNum);
-    setSchema(src.getPlan().getOutSchema());
-  }
-
-  public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId, PartitionType partitionType, int partNum) {
-    this(srcId, targetId, partitionType);
-    this.partitionNum = partNum;
-  }
-
-  public DataChannel(DataChannelProto proto) {
-    this.srcId = new ExecutionBlockId(proto.getSrcId());
-    this.targetId = new ExecutionBlockId(proto.getTargetId());
-    this.transmitType = proto.getTransmitType();
-    this.partitionType = proto.getPartitionType();
-    if (proto.hasSchema()) {
-      this.setSchema(new Schema(proto.getSchema()));
-    }
-    if (proto.getPartitionKeyCount() > 0) {
-      key = new Column[proto.getPartitionKeyCount()];
-      for (int i = 0; i < proto.getPartitionKeyCount(); i++) {
-        key[i] = new Column(proto.getPartitionKey(i));
-      }
-    } else {
-      key = new Column[] {};
-    }
-    if (proto.hasPartitionNum()) {
-      this.partitionNum = proto.getPartitionNum();
-    }
-  }
-
-  public ExecutionBlockId getSrcId() {
-    return srcId;
-  }
-
-  public ExecutionBlockId getTargetId() {
-    return targetId;
-  }
-
-  public PartitionType getPartitionType() {
-    return partitionType;
-  }
-
-  public TransmitType getTransmitType() {
-    return this.transmitType;
-  }
-
-  public void setTransmitType(TransmitType transmitType) {
-    this.transmitType = transmitType;
-  }
-
-  public void setPartition(PartitionType partitionType, Column [] keys, int numPartitions) {
-    Preconditions.checkArgument(keys.length >= 0, "At least one partition key must be specified.");
-    Preconditions.checkArgument(numPartitions > 0, "The number of partitions must be positive: %s", numPartitions);
-
-    this.partitionType = partitionType;
-    this.key = keys;
-    this.partitionNum = numPartitions;
-  }
-
-  public void setPartitionType(PartitionType partitionType) {
-    this.partitionType = partitionType;
-  }
-
-  public boolean hasPartitionKey() {
-    return key != null;
-  }
-
-  public void setPartitionKey(Column [] key) {
-    this.key = key;
-  }
-
-  public Column [] getPartitionKey() {
-    return this.key;
-  }
-
-  public void setPartitionNum(int partNum) {
-    this.partitionNum = partNum;
-  }
-
-  public int getPartitionNum() {
-    return partitionNum;
-  }
-
-  public boolean hasStoreType() {
-    return this.storeType != null;
-  }
-
-  public void setStoreType(StoreType storeType) {
-    this.storeType = storeType;
-  }
-
-  public StoreType getStoreType() {
-    return storeType;
-  }
-
-  public DataChannelProto getProto() {
-    DataChannelProto.Builder builder = DataChannelProto.newBuilder();
-    builder.setSrcId(srcId.getProto());
-    builder.setTargetId(targetId.getProto());
-    if (transmitType != null) {
-      builder.setTransmitType(transmitType);
-    }
-    builder.setPartitionType(partitionType);
-    if (schema != null) {
-      builder.setSchema(schema.getProto());
-    }
-    if (key != null) {
-      for (Column column : key) {
-        builder.addPartitionKey(column.getProto());
-      }
-    }
-    if (partitionNum != null) {
-      builder.setPartitionNum(partitionNum);
-    }
-    return builder.build();
-  }
-
-  public void setSchema(Schema schema) {
-    this.schema = (Schema) schema.clone();
-  }
-
-  public Schema getSchema() {
-    return schema;
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("[").append(srcId.getQueryId()).append("] ");
-    sb.append(srcId.getId()).append(" => ").append(targetId.getId());
-    sb.append(" (type=").append(partitionType);
-    if (hasPartitionKey()) {
-      sb.append(", key=");
-      boolean first = true;
-      for (Column column : getPartitionKey()) {
-        if (first) {
-          first = false;
-        } else {
-          sb.append(",");
-        }
-        sb.append(column.getColumnName());
-      }
-      sb.append(", num=").append(partitionNum);
-    }
-    sb.append(")");
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java
deleted file mode 100644
index 32c06cc..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo;
-
-import com.google.common.base.Objects;
-import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.TajoProtos.TaskAttemptState;
-import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.enforce.Enforcer;
-import org.apache.tajo.storage.Fragment;
-
-import java.io.File;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.CountDownLatch;
-
-
-/**
- * Contains the information about executing subquery.
- */
-public class TaskAttemptContext {
-  private static final Log LOG = LogFactory.getLog(TaskAttemptContext.class);
-  private final TajoConf conf;
-  private final Map<String, List<Fragment>> fragmentMap = new HashMap<String, List<Fragment>>();
-
-  private TaskAttemptState state;
-  private TableStat resultStats;
-  private QueryUnitAttemptId queryId;
-  private final Path workDir;
-  private boolean needFetch = false;
-  private CountDownLatch doneFetchPhaseSignal;
-  private float progress = 0;
-  private Map<Integer, String> repartitions;
-  private File fetchIn;
-  private boolean stopped = false;
-  private boolean interQuery = false;
-  private Path outputPath;
-  private DataChannel dataChannel;
-  private Enforcer enforcer;
-
-  public TaskAttemptContext(TajoConf conf, final QueryUnitAttemptId queryId,
-                            final Fragment[] fragments,
-                            final Path workDir) {
-    this.conf = conf;
-    this.queryId = queryId;
-    
-    for(Fragment t : fragments) {
-      if (fragmentMap.containsKey(t.getName())) {
-        fragmentMap.get(t.getName()).add(t);
-      } else {
-        List<Fragment> frags = new ArrayList<Fragment>();
-        frags.add(t);
-        fragmentMap.put(t.getName(), frags);
-      }
-    }
-
-    this.workDir = workDir;
-    this.repartitions = Maps.newHashMap();
-    
-    state = TaskAttemptState.TA_PENDING;
-  }
-
-  public TajoConf getConf() {
-    return this.conf;
-  }
-  
-  public TaskAttemptState getState() {
-    return this.state;
-  }
-  
-  public void setState(TaskAttemptState state) {
-    this.state = state;
-    LOG.info("Query status of " + getTaskId() + " is changed to " + state);
-  }
-
-  public void setDataChannel(DataChannel dataChannel) {
-    this.dataChannel = dataChannel;
-  }
-
-  public DataChannel getDataChannel() {
-    return dataChannel;
-  }
-
-  public void setEnforcer(Enforcer enforcer) {
-    this.enforcer = enforcer;
-  }
-
-  public Enforcer getEnforcer() {
-    return this.enforcer;
-  }
-
-  public boolean hasResultStats() {
-    return resultStats != null;
-  }
-
-  public void setResultStats(TableStat stats) {
-    this.resultStats = stats;
-  }
-
-  public TableStat getResultStats() {
-    return this.resultStats;
-  }
-  
-  public boolean isStopped() {
-    return this.stopped;
-  }
-
-  public void setInterQuery() {
-    this.interQuery = true;
-  }
-
-  public void setOutputPath(Path outputPath) {
-    this.outputPath = outputPath;
-  }
-
-  public Path getOutputPath() {
-    return this.outputPath;
-  }
-
-  public boolean isInterQuery() {
-    return this.interQuery;
-  }
-  
-  public void stop() {
-    this.stopped = true;
-  }
-  
-  public void addFetchPhase(int count, File fetchIn) {
-    this.needFetch = true;
-    this.doneFetchPhaseSignal = new CountDownLatch(count);
-    this.fetchIn = fetchIn;
-  }
-  
-  public File getFetchIn() {
-    return this.fetchIn;
-  }
-  
-  public boolean hasFetchPhase() {
-    return this.needFetch;
-  }
-  
-  public CountDownLatch getFetchLatch() {
-    return doneFetchPhaseSignal;
-  }
-  
-  public void addRepartition(int partKey, String path) {
-    repartitions.put(partKey, path);
-  }
-  
-  public Iterator<Entry<Integer,String>> getRepartitions() {
-    return repartitions.entrySet().iterator();
-  }
-  
-  public void changeFragment(String tableId, Fragment [] fragments) {
-    fragmentMap.remove(tableId);
-    for(Fragment t : fragments) {
-      if (fragmentMap.containsKey(t.getName())) {
-        fragmentMap.get(t.getName()).add(t);
-      } else {
-        List<Fragment> frags = new ArrayList<Fragment>();
-        frags.add(t);
-        fragmentMap.put(t.getName(), frags);
-      }
-    }
-  }
-  
-  public Path getWorkDir() {
-    return this.workDir;
-  }
-  
-  public QueryUnitAttemptId getTaskId() {
-    return this.queryId;
-  }
-  
-  public float getProgress() {
-    return this.progress;
-  }
-  
-  public void setProgress(float progress) {
-    this.progress = progress;
-  }
-
-  public Fragment getTable(String id) {
-    return fragmentMap.get(id).get(0);
-  }
-
-  public int getFragmentSize() {
-    return fragmentMap.size();
-  }
-
-  public Collection<String> getInputTables() {
-    return fragmentMap.keySet();
-  }
-  
-  public Fragment [] getTables(String id) {
-    return fragmentMap.get(id).toArray(new Fragment[fragmentMap.get(id).size()]);
-  }
-  
-  public int hashCode() {
-    return Objects.hashCode(queryId);
-  }
-  
-  public boolean equals(Object obj) {
-    if (obj instanceof TaskAttemptContext) {
-      TaskAttemptContext other = (TaskAttemptContext) obj;
-      return queryId.equals(other.getTaskId());
-    } else {
-      return false;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetImpl.java
new file mode 100644
index 0000000..b19bb9d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetImpl.java
@@ -0,0 +1,2173 @@
+/**
+ * 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.client;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.storage.Fragment;
+import org.apache.tajo.storage.MergeScanner;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.URL;
+import java.sql.*;
+import java.sql.Date;
+import java.util.*;
+
+public class ResultSetImpl implements ResultSet {
+  private final String cursorName = "tajo";
+  private FileSystem fs;
+  private Scanner scanner;
+  private TableDesc desc;
+  private Schema schema;
+  private Tuple cur;
+  private int curRow;
+  private long totalRow;
+  private boolean wasNull;
+  private TajoClient tajoClient;
+  QueryId queryId;
+
+  public ResultSetImpl(TajoClient tajoClient, QueryId queryId) {
+    this.tajoClient = tajoClient;
+    this.queryId = queryId;
+    init();
+  }
+
+  public ResultSetImpl(TajoClient tajoClient, QueryId queryId, Configuration conf, TableDesc desc) throws IOException {
+    this.tajoClient = tajoClient;
+    this.queryId = queryId;
+    this.desc = desc;
+    this.schema = desc.getMeta().getSchema();
+    if(desc != null) {
+      fs = desc.getPath().getFileSystem(conf);
+      this.totalRow = desc.getMeta().getStat() != null ? desc.getMeta().getStat().getNumRows() : 0;
+      Collection<Fragment> frags = getFragments(desc.getMeta(), desc.getPath());
+      scanner = new MergeScanner(conf, desc.getMeta(), frags);
+    }
+    init();
+  }
+
+  private void init() {
+    cur = null;
+    curRow = 0;
+  }
+
+  class FileNameComparator implements Comparator<FileStatus> {
+
+    @Override
+    public int compare(FileStatus f1, FileStatus f2) {
+      return f2.getPath().getName().compareTo(f1.getPath().getName());
+    }
+  }
+
+  private Collection<Fragment> getFragments(TableMeta meta, Path tablePath)
+      throws IOException {
+    List<Fragment> fraglist = Lists.newArrayList();
+    FileStatus[] files = fs.listStatus(tablePath, new PathFilter() {
+      @Override
+      public boolean accept(Path path) {
+        return path.getName().charAt(0) != '.';
+      }
+    });
+    Arrays.sort(files, new FileNameComparator());
+
+    String tbname = tablePath.getName();
+    for (int i = 0; i < files.length; i++) {
+      if (files[i].getLen() == 0) {
+        continue;
+      }
+      fraglist.add(new Fragment(tbname + "_" + i, files[i].getPath(), meta, 0l, files[i].getLen()));
+    }
+    return fraglist;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.Wrapper#isWrapperFor(java.lang.Class)
+   */
+  @Override
+  public boolean isWrapperFor(Class<?> arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.Wrapper#unwrap(java.lang.Class)
+   */
+  @Override
+  public <T> T unwrap(Class<T> arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#absolute(int)
+   */
+  @Override
+  public boolean absolute(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#afterLast()
+   */
+  @Override
+  public void afterLast() throws SQLException {
+    while (this.next())
+      ;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#beforeFirst()
+   */
+  @Override
+  public void beforeFirst() throws SQLException {
+    try {
+      if(scanner != null) {
+        scanner.reset();
+      }
+      init();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#cancelRowUpdates()
+   */
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#clearWarnings()
+   */
+  @Override
+  public void clearWarnings() throws SQLException {
+    // TODO
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#close()
+   */
+  @Override
+  public void close() throws SQLException {
+    try {
+      if(tajoClient != null) {
+        this.tajoClient.closeQuery(queryId);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+    try {
+      if(scanner != null) {
+        this.scanner.close();
+      }
+      //TODO clean temp result file
+      cur = null;
+      curRow = -1;
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#deleteRow()
+   */
+  @Override
+  public void deleteRow() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#findColumn(java.lang.String)
+   */
+  @Override
+  public int findColumn(String colName) throws SQLException {
+    return schema.getColumnIdByName(colName);
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#first()
+   */
+  @Override
+  public boolean first() throws SQLException {
+    this.beforeFirst();
+    return this.next();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getArray(int)
+   */
+  @Override
+  public Array getArray(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getArray(java.lang.String)
+   */
+  @Override
+  public Array getArray(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getAsciiStream(int)
+   */
+  @Override
+  public InputStream getAsciiStream(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getAsciiStream(java.lang.String)
+   */
+  @Override
+  public InputStream getAsciiStream(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBigDecimal(int)
+   */
+  @Override
+  public BigDecimal getBigDecimal(int fieldId) throws SQLException {
+    // TODO
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBigDecimal(java.lang.String)
+   */
+  @Override
+  public BigDecimal getBigDecimal(String fieldName) throws SQLException {
+    // TODO
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBigDecimal(int, int)
+   */
+  @Override
+  public BigDecimal getBigDecimal(int arg0, int arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBigDecimal(java.lang.String, int)
+   */
+  @Override
+  public BigDecimal getBigDecimal(String arg0, int arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBinaryStream(int)
+   */
+  @Override
+  public InputStream getBinaryStream(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBinaryStream(java.lang.String)
+   */
+  @Override
+  public InputStream getBinaryStream(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBlob(int)
+   */
+  @Override
+  public Blob getBlob(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBlob(java.lang.String)
+   */
+  @Override
+  public Blob getBlob(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBoolean(int)
+   */
+  @Override
+  public boolean getBoolean(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asBool();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBoolean(java.lang.String)
+   */
+  @Override
+  public boolean getBoolean(String colName) throws SQLException {
+    Datum datum = cur.get(findColumn(colName));
+    handleNull(datum);
+    return datum.asBool();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getByte(int)
+   */
+  @Override
+  public byte getByte(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asByte();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getByte(java.lang.String)
+   */
+  @Override
+  public byte getByte(String name) throws SQLException {
+    Datum datum = cur.get(findColumn(name));
+    handleNull(datum);
+    return datum.asByte();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBytes(int)
+   */
+  @Override
+  public byte[] getBytes(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asByteArray();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getBytes(java.lang.String)
+   */
+  @Override
+  public byte[] getBytes(String name) throws SQLException {
+    Datum datum = cur.get(findColumn(name));
+    handleNull(datum);
+    return datum.asByteArray();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getCharacterStream(int)
+   */
+  @Override
+  public Reader getCharacterStream(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getCharacterStream(java.lang.String)
+   */
+  @Override
+  public Reader getCharacterStream(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getClob(int)
+   */
+  @Override
+  public Clob getClob(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getClob(java.lang.String)
+   */
+  @Override
+  public Clob getClob(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getConcurrency()
+   */
+  @Override
+  public int getConcurrency() throws SQLException {
+    return ResultSet.CONCUR_READ_ONLY;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getCursorName()
+   */
+  @Override
+  public String getCursorName() throws SQLException {
+    return cursorName;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getDate(int)
+   */
+  @Override
+  public Date getDate(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getDate(java.lang.String)
+   */
+  @Override
+  public Date getDate(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getDate(int, java.util.Calendar)
+   */
+  @Override
+  public Date getDate(int arg0, Calendar arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getDate(java.lang.String, java.util.Calendar)
+   */
+  @Override
+  public Date getDate(String arg0, Calendar arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getDouble(int)
+   */
+  @Override
+  public double getDouble(int fieldId) throws SQLException {
+    Datum datum = cur.getDouble(fieldId - 1);
+    handleNull(datum);
+    return datum.asFloat8();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getDouble(java.lang.String)
+   */
+  @Override
+  public double getDouble(String name) throws SQLException {
+    Datum datum = cur.get(findColumn(name));
+    handleNull(datum);
+    return datum.asFloat8();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getFetchDirection()
+   */
+  @Override
+  public int getFetchDirection() throws SQLException {
+    return ResultSet.FETCH_FORWARD;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getFetchSize()
+   */
+  @Override
+  public int getFetchSize() throws SQLException {
+    return 0;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getFloat(int)
+   */
+  @Override
+  public float getFloat(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asFloat4();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getFloat(java.lang.String)
+   */
+  @Override
+  public float getFloat(String name) throws SQLException {
+    Datum datum = cur.get(findColumn(name));
+    handleNull(datum);
+    return datum.asFloat4();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getHoldability()
+   */
+  @Override
+  public int getHoldability() throws SQLException {
+    return 0;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getInt(int)
+   */
+  @Override
+  public int getInt(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asInt4();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getInt(java.lang.String)
+   */
+  @Override
+  public int getInt(String name) throws SQLException {
+    Datum datum = cur.get(findColumn(name));
+    handleNull(datum);
+    return datum.asInt4();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getLong(int)
+   */
+  @Override
+  public long getLong(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asInt8();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getLong(java.lang.String)
+   */
+  @Override
+  public long getLong(String name) throws SQLException {
+    Datum datum = cur.get(findColumn(name));
+    handleNull(datum);
+    return datum.asInt8();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getMetaData()
+   */
+  @Override
+  public ResultSetMetaData getMetaData() throws SQLException {
+    return new ResultSetMetaDataImpl(desc.getMeta());
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getNCharacterStream(int)
+   */
+  @Override
+  public Reader getNCharacterStream(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getNCharacterStream(java.lang.String)
+   */
+  @Override
+  public Reader getNCharacterStream(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getNClob(int)
+   */
+  @Override
+  public NClob getNClob(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getNClob(java.lang.String)
+   */
+  @Override
+  public NClob getNClob(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getNString(int)
+   */
+  @Override
+  public String getNString(int fieldId) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getNString(java.lang.String)
+   */
+  @Override
+  public String getNString(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getObject(int)
+   */
+  @Override
+  public Object getObject(int fieldId) throws SQLException {
+    Datum d = cur.get(fieldId - 1);
+    handleNull(d);
+
+    // TODO - to be changed to return Object type
+    return d;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getObject(java.lang.String)
+   */
+  @Override
+  public Object getObject(String arg0) throws SQLException {
+    Datum d = cur.get(findColumn(arg0));
+    handleNull(d);
+    return d;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getObject(int, java.util.Map)
+   */
+  @Override
+  public Object getObject(int arg0, Map<String, Class<?>> arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getObject(java.lang.String, java.util.Map)
+   */
+  @Override
+  public Object getObject(String arg0, Map<String, Class<?>> arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getObject(java.lang.String, java.lang.Class)
+   */
+  public <T> T getObject(String arg0, Class<T> arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getObject(int, java.lang.Class)
+   */
+  public <T> T getObject(int arg0, Class<T> arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getRef(int)
+   */
+  @Override
+  public Ref getRef(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getRef(java.lang.String)
+   */
+  @Override
+  public Ref getRef(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getRow()
+   */
+  @Override
+  public int getRow() throws SQLException {
+    return curRow;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getRowId(int)
+   */
+  @Override
+  public RowId getRowId(int fieldId) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getRowId(java.lang.String)
+   */
+  @Override
+  public RowId getRowId(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getSQLXML(int)
+   */
+  @Override
+  public SQLXML getSQLXML(int fieldId) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getSQLXML(java.lang.String)
+   */
+  @Override
+  public SQLXML getSQLXML(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getShort(int)
+   */
+  @Override
+  public short getShort(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asInt2();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getShort(java.lang.String)
+   */
+  @Override
+  public short getShort(String name) throws SQLException {
+    Datum datum = cur.get(findColumn(name));
+    handleNull(datum);
+    return datum.asInt2();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getStatement()
+   */
+  @Override
+  public Statement getStatement() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getString(int)
+   */
+  @Override
+  public String getString(int fieldId) throws SQLException {
+    Datum datum = cur.get(fieldId - 1);
+    handleNull(datum);
+    return datum.asChars();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getString(java.lang.String)
+   */
+  @Override
+  public String getString(String arg0) throws SQLException {
+    Datum datum = cur.get(findColumn(arg0));
+    handleNull(datum);
+    return datum.asChars();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTime(int)
+   */
+  @Override
+  public Time getTime(int fieldId) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTime(java.lang.String)
+   */
+  @Override
+  public Time getTime(String name) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTime(int, java.util.Calendar)
+   */
+  @Override
+  public Time getTime(int fieldId, Calendar arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTime(java.lang.String, java.util.Calendar)
+   */
+  @Override
+  public Time getTime(String name, Calendar arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTimestamp(int)
+   */
+  @Override
+  public Timestamp getTimestamp(int fieldId) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTimestamp(java.lang.String)
+   */
+  @Override
+  public Timestamp getTimestamp(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTimestamp(int, java.util.Calendar)
+   */
+  @Override
+  public Timestamp getTimestamp(int fieldId, Calendar arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getTimestamp(java.lang.String, java.util.Calendar)
+   */
+  @Override
+  public Timestamp getTimestamp(String arg0, Calendar arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getType()
+   */
+  @Override
+  public int getType() throws SQLException {
+    return ResultSet.TYPE_FORWARD_ONLY;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getURL(int)
+   */
+  @Override
+  public URL getURL(int fieldId) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getURL(java.lang.String)
+   */
+  @Override
+  public URL getURL(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getUnicodeStream(int)
+   */
+  @Override
+  public InputStream getUnicodeStream(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getUnicodeStream(java.lang.String)
+   */
+  @Override
+  public InputStream getUnicodeStream(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#getWarnings()
+   */
+  @Override
+  public SQLWarning getWarnings() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#insertRow()
+   */
+  @Override
+  public void insertRow() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#isAfterLast()
+   */
+  @Override
+  public boolean isAfterLast() throws SQLException {
+    return this.curRow > this.totalRow;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#isBeforeFirst()
+   */
+  @Override
+  public boolean isBeforeFirst() throws SQLException {
+    return this.curRow == 0;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#isClosed()
+   */
+  @Override
+  public boolean isClosed() throws SQLException {
+    return this.curRow == -1;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#isFirst()
+   */
+  @Override
+  public boolean isFirst() throws SQLException {
+    return this.curRow == 1;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#isLast()
+   */
+  @Override
+  public boolean isLast() throws SQLException {
+    return this.curRow == this.totalRow;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#last()
+   */
+  @Override
+  public boolean last() throws SQLException {
+    Tuple last = null;
+    while (this.next()) {
+      last = cur;
+    }
+    cur = last;
+    return true;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#moveToCurrentRow()
+   */
+  @Override
+  public void moveToCurrentRow() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#moveToInsertRow()
+   */
+  @Override
+  public void moveToInsertRow() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#next()
+   */
+  @Override
+  public boolean next() throws SQLException {
+    if(scanner == null) {
+      return false;
+    }
+    try {
+      if (totalRow <= 0)
+        return false;
+
+      cur = scanner.next();
+      curRow++;
+      if (cur != null) {
+        return true;
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    return false;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#previous()
+   */
+  @Override
+  public boolean previous() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#refreshRow()
+   */
+  @Override
+  public void refreshRow() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#relative(int)
+   */
+  @Override
+  public boolean relative(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#rowDeleted()
+   */
+  @Override
+  public boolean rowDeleted() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#rowInserted()
+   */
+  @Override
+  public boolean rowInserted() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#rowUpdated()
+   */
+  @Override
+  public boolean rowUpdated() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#setFetchDirection(int)
+   */
+  @Override
+  public void setFetchDirection(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#setFetchSize(int)
+   */
+  @Override
+  public void setFetchSize(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateArray(int, java.sql.Array)
+   */
+  @Override
+  public void updateArray(int arg0, Array arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateArray(java.lang.String, java.sql.Array)
+   */
+  @Override
+  public void updateArray(String arg0, Array arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateAsciiStream(int, java.io.InputStream)
+   */
+  @Override
+  public void updateAsciiStream(int arg0, InputStream arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateAsciiStream(java.lang.String,
+   * java.io.InputStream)
+   */
+  @Override
+  public void updateAsciiStream(String arg0, InputStream arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateAsciiStream(int, java.io.InputStream, int)
+   */
+  @Override
+  public void updateAsciiStream(int arg0, InputStream arg1, int arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateAsciiStream(java.lang.String,
+   * java.io.InputStream, int)
+   */
+  @Override
+  public void updateAsciiStream(String arg0, InputStream arg1, int arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateAsciiStream(int, java.io.InputStream, long)
+   */
+  @Override
+  public void updateAsciiStream(int arg0, InputStream arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateAsciiStream(java.lang.String,
+   * java.io.InputStream, long)
+   */
+  @Override
+  public void updateAsciiStream(String arg0, InputStream arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBigDecimal(int, java.math.BigDecimal)
+   */
+  @Override
+  public void updateBigDecimal(int arg0, BigDecimal arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBigDecimal(java.lang.String,
+   * java.math.BigDecimal)
+   */
+  @Override
+  public void updateBigDecimal(String arg0, BigDecimal arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBinaryStream(int, java.io.InputStream)
+   */
+  @Override
+  public void updateBinaryStream(int arg0, InputStream arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBinaryStream(java.lang.String,
+   * java.io.InputStream)
+   */
+  @Override
+  public void updateBinaryStream(String arg0, InputStream arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBinaryStream(int, java.io.InputStream, int)
+   */
+  @Override
+  public void updateBinaryStream(int arg0, InputStream arg1, int arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBinaryStream(java.lang.String,
+   * java.io.InputStream, int)
+   */
+  @Override
+  public void updateBinaryStream(String arg0, InputStream arg1, int arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBinaryStream(int, java.io.InputStream, long)
+   */
+  @Override
+  public void updateBinaryStream(int arg0, InputStream arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBinaryStream(java.lang.String,
+   * java.io.InputStream, long)
+   */
+  @Override
+  public void updateBinaryStream(String arg0, InputStream arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBlob(int, java.sql.Blob)
+   */
+  @Override
+  public void updateBlob(int arg0, Blob arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBlob(java.lang.String, java.sql.Blob)
+   */
+  @Override
+  public void updateBlob(String arg0, Blob arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBlob(int, java.io.InputStream)
+   */
+  @Override
+  public void updateBlob(int arg0, InputStream arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBlob(java.lang.String, java.io.InputStream)
+   */
+  @Override
+  public void updateBlob(String arg0, InputStream arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBlob(int, java.io.InputStream, long)
+   */
+  @Override
+  public void updateBlob(int arg0, InputStream arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBlob(java.lang.String, java.io.InputStream,
+   * long)
+   */
+  @Override
+  public void updateBlob(String arg0, InputStream arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBoolean(int, boolean)
+   */
+  @Override
+  public void updateBoolean(int arg0, boolean arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBoolean(java.lang.String, boolean)
+   */
+  @Override
+  public void updateBoolean(String arg0, boolean arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateByte(int, byte)
+   */
+  @Override
+  public void updateByte(int arg0, byte arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateByte(java.lang.String, byte)
+   */
+  @Override
+  public void updateByte(String arg0, byte arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBytes(int, byte[])
+   */
+  @Override
+  public void updateBytes(int arg0, byte[] arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateBytes(java.lang.String, byte[])
+   */
+  @Override
+  public void updateBytes(String arg0, byte[] arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateCharacterStream(int, java.io.Reader)
+   */
+  @Override
+  public void updateCharacterStream(int arg0, Reader arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateCharacterStream(java.lang.String,
+   * java.io.Reader)
+   */
+  @Override
+  public void updateCharacterStream(String arg0, Reader arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateCharacterStream(int, java.io.Reader, int)
+   */
+  @Override
+  public void updateCharacterStream(int arg0, Reader arg1, int arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateCharacterStream(java.lang.String,
+   * java.io.Reader, int)
+   */
+  @Override
+  public void updateCharacterStream(String arg0, Reader arg1, int arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateCharacterStream(int, java.io.Reader, long)
+   */
+  @Override
+  public void updateCharacterStream(int arg0, Reader arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateCharacterStream(java.lang.String,
+   * java.io.Reader, long)
+   */
+  @Override
+  public void updateCharacterStream(String arg0, Reader arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateClob(int, java.sql.Clob)
+   */
+  @Override
+  public void updateClob(int arg0, Clob arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateClob(java.lang.String, java.sql.Clob)
+   */
+  @Override
+  public void updateClob(String arg0, Clob arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateClob(int, java.io.Reader)
+   */
+  @Override
+  public void updateClob(int arg0, Reader arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateClob(java.lang.String, java.io.Reader)
+   */
+  @Override
+  public void updateClob(String arg0, Reader arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateClob(int, java.io.Reader, long)
+   */
+  @Override
+  public void updateClob(int arg0, Reader arg1, long arg2) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateClob(java.lang.String, java.io.Reader, long)
+   */
+  @Override
+  public void updateClob(String arg0, Reader arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateDate(int, java.sql.Date)
+   */
+  @Override
+  public void updateDate(int arg0, Date arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateDate(java.lang.String, java.sql.Date)
+   */
+  @Override
+  public void updateDate(String arg0, Date arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateDouble(int, double)
+   */
+  @Override
+  public void updateDouble(int arg0, double arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateDouble(java.lang.String, double)
+   */
+  @Override
+  public void updateDouble(String arg0, double arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateFloat(int, float)
+   */
+  @Override
+  public void updateFloat(int arg0, float arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateFloat(java.lang.String, float)
+   */
+  @Override
+  public void updateFloat(String arg0, float arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateInt(int, int)
+   */
+  @Override
+  public void updateInt(int arg0, int arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateInt(java.lang.String, int)
+   */
+  @Override
+  public void updateInt(String arg0, int arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateLong(int, long)
+   */
+  @Override
+  public void updateLong(int arg0, long arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateLong(java.lang.String, long)
+   */
+  @Override
+  public void updateLong(String arg0, long arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNCharacterStream(int, java.io.Reader)
+   */
+  @Override
+  public void updateNCharacterStream(int arg0, Reader arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNCharacterStream(java.lang.String,
+   * java.io.Reader)
+   */
+  @Override
+  public void updateNCharacterStream(String arg0, Reader arg1)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNCharacterStream(int, java.io.Reader, long)
+   */
+  @Override
+  public void updateNCharacterStream(int arg0, Reader arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNCharacterStream(java.lang.String,
+   * java.io.Reader, long)
+   */
+  @Override
+  public void updateNCharacterStream(String arg0, Reader arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNClob(int, java.sql.NClob)
+   */
+  @Override
+  public void updateNClob(int arg0, NClob arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNClob(java.lang.String, java.sql.NClob)
+   */
+  @Override
+  public void updateNClob(String arg0, NClob arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNClob(int, java.io.Reader)
+   */
+  @Override
+  public void updateNClob(int arg0, Reader arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNClob(java.lang.String, java.io.Reader)
+   */
+  @Override
+  public void updateNClob(String arg0, Reader arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNClob(int, java.io.Reader, long)
+   */
+  @Override
+  public void updateNClob(int arg0, Reader arg1, long arg2) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNClob(java.lang.String, java.io.Reader, long)
+   */
+  @Override
+  public void updateNClob(String arg0, Reader arg1, long arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNString(int, java.lang.String)
+   */
+  @Override
+  public void updateNString(int arg0, String arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNString(java.lang.String, java.lang.String)
+   */
+  @Override
+  public void updateNString(String arg0, String arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNull(int)
+   */
+  @Override
+  public void updateNull(int arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateNull(java.lang.String)
+   */
+  @Override
+  public void updateNull(String arg0) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateObject(int, java.lang.Object)
+   */
+  @Override
+  public void updateObject(int arg0, Object arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateObject(java.lang.String, java.lang.Object)
+   */
+  @Override
+  public void updateObject(String arg0, Object arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateObject(int, java.lang.Object, int)
+   */
+  @Override
+  public void updateObject(int arg0, Object arg1, int arg2) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateObject(java.lang.String, java.lang.Object,
+   * int)
+   */
+  @Override
+  public void updateObject(String arg0, Object arg1, int arg2)
+      throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateRef(int, java.sql.Ref)
+   */
+  @Override
+  public void updateRef(int arg0, Ref arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateRef(java.lang.String, java.sql.Ref)
+   */
+  @Override
+  public void updateRef(String arg0, Ref arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateRow()
+   */
+  @Override
+  public void updateRow() throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateRowId(int, java.sql.RowId)
+   */
+  @Override
+  public void updateRowId(int arg0, RowId arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateRowId(java.lang.String, java.sql.RowId)
+   */
+  @Override
+  public void updateRowId(String arg0, RowId arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateSQLXML(int, java.sql.SQLXML)
+   */
+  @Override
+  public void updateSQLXML(int arg0, SQLXML arg1) throws SQLException {
+    throw new UnsupportedException();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateSQLXML(java.lang.String, java.sql.SQLXML)
+   */
+  @Override
+  public void updateSQLXML(String arg0, SQLXML arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateShort(int, short)
+   */
+  @Override
+  public void updateShort(int arg0, short arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateShort(java.lang.String, short)
+   */
+  @Override
+  public void updateShort(String arg0, short arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateString(int, java.lang.String)
+   */
+  @Override
+  public void updateString(int arg0, String arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateString(java.lang.String, java.lang.String)
+   */
+  @Override
+  public void updateString(String arg0, String arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateTime(int, java.sql.Time)
+   */
+  @Override
+  public void updateTime(int arg0, Time arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateTime(java.lang.String, java.sql.Time)
+   */
+  @Override
+  public void updateTime(String arg0, Time arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateTimestamp(int, java.sql.Timestamp)
+   */
+  @Override
+  public void updateTimestamp(int arg0, Timestamp arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#updateTimestamp(java.lang.String,
+   * java.sql.Timestamp)
+   */
+  @Override
+  public void updateTimestamp(String arg0, Timestamp arg1) throws SQLException {
+    throw new UnsupportedException();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.sql.ResultSet#wasNull()
+   */
+  @Override
+  public boolean wasNull() throws SQLException {
+    return wasNull;
+  }
+
+  private void handleNull(Datum d) {
+    wasNull = (d instanceof NullDatum);
+  }
+
+  public boolean hasResult() {
+    return scanner != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetMetaDataImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetMetaDataImpl.java
new file mode 100644
index 0000000..d8c8ad1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/ResultSetMetaDataImpl.java
@@ -0,0 +1,259 @@
+/**
+ * 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.client;
+
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.exception.UnsupportedException;
+
+import java.nio.channels.UnsupportedAddressTypeException;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Types;
+
+public class ResultSetMetaDataImpl implements ResultSetMetaData {
+  private TableMeta meta;
+  
+  public ResultSetMetaDataImpl(TableMeta meta) {
+    this.meta = meta;
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.Wrapper#isWrapperFor(java.lang.Class)
+   */
+  @Override
+  public boolean isWrapperFor(Class<?> arg0) throws SQLException {
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.Wrapper#unwrap(java.lang.Class)
+   */
+  @Override
+  public <T> T unwrap(Class<T> arg0) throws SQLException {
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getCatalogName(int)
+   */
+  @Override
+  public String getCatalogName(int column) throws SQLException {
+    // TODO Auto-generated method stub
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getColumnClassName(int)
+   */
+  @Override
+  public String getColumnClassName(int column) throws SQLException {
+    return meta.getSchema().getColumn(column - 1).getClass().getName();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getColumnCount()
+   */
+  @Override
+  public int getColumnCount() throws SQLException {
+    return meta.getSchema().getColumnNum();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getColumnDisplaySize(int)
+   */
+  @Override
+  public int getColumnDisplaySize(int column) throws SQLException {
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getColumnLabel(int)
+   */
+  @Override
+  public String getColumnLabel(int column) throws SQLException {
+    return meta.getSchema().getColumn(column - 1).getQualifiedName();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getColumnName(int)
+   */
+  @Override
+  public String getColumnName(int column) throws SQLException {
+    return meta.getSchema().getColumn(column - 1).getColumnName();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getColumnType(int)
+   */
+  @Override
+  public int getColumnType(int column) throws SQLException {
+    // TODO
+    DataType type = meta.getSchema().getColumn(column - 1).getDataType();
+    switch (type.getType()) {
+      case BOOLEAN:
+        return Types.BOOLEAN;
+      case INT1:
+        return Types.TINYINT;
+      case INT2:
+        return Types.SMALLINT;
+      case INT4:
+        return Types.INTEGER;
+      case INT8:
+        return Types.BIGINT;
+      case FLOAT4:
+        return Types.FLOAT;
+      case FLOAT8:
+        return Types.DOUBLE;
+      case DECIMAL:
+        return Types.DECIMAL;
+      case VARBINARY:
+        return Types.VARBINARY;
+      case CHAR:
+        return Types.CHAR;
+      case DATE:
+        return Types.DATE;
+      case VARCHAR:
+        return Types.VARCHAR;
+      case TEXT:
+        return Types.VARCHAR;
+      default:
+        throw new UnsupportedException();
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getColumnTypeName(int)
+   */
+  @Override
+  public String getColumnTypeName(int column) throws SQLException {
+    return meta.getSchema().getColumn(column - 1).
+        getDataType().getClass().getCanonicalName();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getPrecision(int)
+   */
+  @Override
+  public int getPrecision(int column) throws SQLException {
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getScale(int)
+   */
+  @Override
+  public int getScale(int column) throws SQLException {
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getSchemaName(int)
+   */
+  @Override
+  public String getSchemaName(int column) throws SQLException {
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#getTableName(int)
+   */
+  @Override
+  public String getTableName(int column) throws SQLException {
+    return meta.getSchema().getColumn(column - 1).getQualifier();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isAutoIncrement(int)
+   */
+  @Override
+  public boolean isAutoIncrement(int column) throws SQLException {
+    return false;
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isCaseSensitive(int)
+   */
+  @Override
+  public boolean isCaseSensitive(int column) throws SQLException {
+    return false;
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isCurrency(int)
+   */
+  @Override
+  public boolean isCurrency(int column) throws SQLException {
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isDefinitelyWritable(int)
+   */
+  @Override
+  public boolean isDefinitelyWritable(int column) throws SQLException {
+    return false;
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isNullable(int)
+   */
+  @Override
+  public int isNullable(int column) throws SQLException {
+    // TODO Auto-generated method stub
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isReadOnly(int)
+   */
+  @Override
+  public boolean isReadOnly(int column) throws SQLException {
+    return true;
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isSearchable(int)
+   */
+  @Override
+  public boolean isSearchable(int column) throws SQLException {
+    // TODO
+    return true;
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isSigned(int)
+   */
+  @Override
+  public boolean isSigned(int column) throws SQLException {
+    // TODO Auto-generated method stub
+    throw new UnsupportedAddressTypeException();
+  }
+
+  /* (non-Javadoc)
+   * @see java.sql.ResultSetMetaData#isWritable(int)
+   */
+  @Override
+  public boolean isWritable(int column) throws SQLException {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
index a819f7a..324ced4 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
@@ -31,7 +31,6 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.engine.query.ResultSetImpl;
 import org.apache.tajo.ipc.ClientProtos.*;
 import org.apache.tajo.ipc.QueryMasterClientProtocol;
 import org.apache.tajo.ipc.QueryMasterClientProtocol.QueryMasterClientProtocolService;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java
new file mode 100644
index 0000000..3125e30
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/AmbiguousFieldException.java
@@ -0,0 +1,30 @@
+/**
+ * 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.engine.exception;
+
+public class AmbiguousFieldException extends InvalidQueryException {
+	private static final long serialVersionUID = 3102675985226352347L;
+
+	/**
+	 * @param fieldName
+	 */
+	public AmbiguousFieldException(String fieldName) {
+		super("ERROR: column name "+ fieldName + " is ambiguous");	
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.java
deleted file mode 100644
index 0c7795c..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.java
+++ /dev/null
@@ -1,34 +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.engine.exception;
-
-public class EmptyClusterException extends Exception {
-
-  public EmptyClusterException() {
-
-  }
-
-  public EmptyClusterException(String msg) {
-    super(msg);
-  }
-
-  public EmptyClusterException(Exception e) {
-    super(e);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3f45f0cd/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.java
new file mode 100644
index 0000000..fba691d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/InvalidQueryException.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.engine.exception;
+
+public class InvalidQueryException extends RuntimeException {
+	private static final long serialVersionUID = -7085849718839416246L;
+
+  public InvalidQueryException() {
+    super();
+  }
+
+	public InvalidQueryException(String message) {
+    super(message);
+  }
+	
+	public InvalidQueryException(Throwable t) {
+		super(t);
+	}
+}

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