You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2014/12/09 02:56:22 UTC

[01/10] tajo git commit: TAJO-1231: Implicit table properties in session is not stored in table property.

Repository: tajo
Updated Branches:
  refs/heads/index_support 72600c138 -> 761e372ba


TAJO-1231: Implicit table properties in session is not stored in table property.

Closes #287


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

Branch: refs/heads/index_support
Commit: ab2efce8fb4015523a9a818f0e03fa57c6366585
Parents: 2a69bcc
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Dec 5 11:43:15 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Dec 5 16:36:12 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  7 ++--
 .../org/apache/tajo/client/TestTajoClient.java  | 37 +++++++++++++++++---
 .../org/apache/tajo/plan/LogicalPlanner.java    | 20 ++++++++---
 .../org/apache/tajo/plan/util/PlannerUtil.java  | 17 +++++++++
 4 files changed, 70 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 0c785ce..8ecd795 100644
--- a/CHANGES
+++ b/CHANGES
@@ -83,8 +83,11 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
-    TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data when result is empty.
-    (jaehwa)
+    TAJO-1231: Implicit table properties in session are not stored in 
+    table property. (hyunsik)
+
+    TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data 
+    when result is empty. (jaehwa)
 
     TAJO-1191: Change DateDatum timezone to UTC. (Jaewoong Jung via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
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 616799f..21dd7d5 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
@@ -680,7 +680,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public void testSetCvsNull() throws Exception {
+  public void testNullCharSession() throws Exception {
     String sql =
         "select\n" +
             "  c_custkey,\n" +
@@ -692,17 +692,44 @@ public class TestTajoClient {
             "  c_custkey,\n" +
             "  orders.o_orderkey;\n";
 
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-
     Map<String, String> variables = new HashMap<String, String>();
     variables.put(SessionVars.NULL_CHAR.keyname(), "\\\\T");
     client.updateSessionVariables(variables);
+    TajoResultSet resultDesc = (TajoResultSet)client.executeQueryAndGetResult(sql);
+    resultDesc.close();
+    assertNullCharSessionVar(resultDesc.getTableDesc());
+  }
+
+  @Test
+  public void testNullCharSessionInCTAS() throws Exception {
+    String sql =
+        "create table nullcharsession as select\n" +
+            "  c_custkey,\n" +
+            "  orders.o_orderkey,\n" +
+            "  orders.o_orderstatus \n" +
+            "from\n" +
+            "  orders full outer join customer on c_custkey = o_orderkey\n" +
+            "order by\n" +
+            "  c_custkey,\n" +
+            "  orders.o_orderkey;\n";
 
+    Map<String, String> variables = new HashMap<String, String>();
+    variables.put(SessionVars.NULL_CHAR.keyname(), "\\\\T");
+    client.updateSessionVariables(variables);
     TajoResultSet res = (TajoResultSet)client.executeQueryAndGetResult(sql);
+    res.close();
+
+    TableDesc resultDesc = client.getTableDesc("nullcharsession");
+    assertNullCharSessionVar(resultDesc);
+  }
+
+
+  public void assertNullCharSessionVar(TableDesc resultDesc) throws Exception {
+    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
 
-    assertEquals(res.getTableDesc().getMeta().getOption(StorageConstants.TEXT_NULL), "\\\\T");
+    assertEquals(resultDesc.getMeta().getOption(StorageConstants.TEXT_NULL), "\\\\T");
 
-    Path path = new Path(res.getTableDesc().getPath());
+    Path path = new Path(resultDesc.getPath());
     FileSystem fs = path.getFileSystem(tajoConf);
 
     FileStatus[] files = fs.listStatus(path);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
index 3b1b4e3..69c0e4b 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
@@ -1706,13 +1706,25 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       createTableNode.setStorageType(CatalogProtos.StoreType.CSV);
     }
 
-    // Set default storage properties to be created.
-    KeyValueSet keyValueSet = CatalogUtil.newPhysicalProperties(createTableNode.getStorageType());
+
+
+    // Set default storage properties to table
+    KeyValueSet properties = CatalogUtil.newPhysicalProperties(createTableNode.getStorageType());
+
+    // Priority to apply table properties
+    // 1. Explicit table properties specified in WITH clause
+    // 2. Session variables
+
+    // Set session variables to properties
+    PlannerUtil.applySessionToTableProperties(context.queryContext, createTableNode.getStorageType(), properties);
+    // Set table properties specified in WITH clause
     if (expr.hasParams()) {
-      keyValueSet.putAll(expr.getParams());
+      properties.putAll(expr.getParams());
     }
 
-    createTableNode.setOptions(keyValueSet);
+    createTableNode.setOptions(properties);
+
+
 
     if (expr.hasPartition()) {
       if (expr.getPartitionMethod().getPartitionType().equals(PartitionType.COLUMN)) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
index 6868b6c..c55c203 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
@@ -21,6 +21,8 @@ package org.apache.tajo.plan.util;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.Column;
@@ -35,10 +37,15 @@ import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor;
 import org.apache.tajo.plan.visitor.ExplainLogicalPlanVisitor;
 import org.apache.tajo.plan.visitor.SimpleAlgebraVisitor;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
 
 import java.util.*;
 
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.CSV;
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.TEXTFILE;
+
 public class PlannerUtil {
 
   public static boolean checkIfDDLPlan(LogicalNode node) {
@@ -776,4 +783,14 @@ public class PlannerUtil {
 
     return explains.toString();
   }
+
+  public static void applySessionToTableProperties(OverridableConf sessionVars,
+                                                   CatalogProtos.StoreType storeType,
+                                                   KeyValueSet tableProperties) {
+    if (storeType == CSV || storeType == TEXTFILE) {
+      if (sessionVars.containsKey(SessionVars.NULL_CHAR)) {
+        tableProperties.set(StorageConstants.TEXT_NULL, sessionVars.get(SessionVars.NULL_CHAR));
+      }
+    }
+  }
 }


[06/10] tajo git commit: TAJO-1229: rename tajo-yarn-pullserver to tajo-pullserver.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
new file mode 100644
index 0000000..5a4e69f
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
@@ -0,0 +1,808 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.pullserver.retriever.FileChunk;
+import org.apache.tajo.rpc.RpcChannelFactory;
+import org.apache.tajo.storage.HashShuffleAppenderManager;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.handler.codec.http.*;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+import java.io.*;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+public class TajoPullServerService extends AbstractService {
+
+  private static final Log LOG = LogFactory.getLog(TajoPullServerService.class);
+
+  public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache";
+  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+  public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes";
+  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+  private int port;
+  private ChannelFactory selector;
+  private final ChannelGroup accepted = new DefaultChannelGroup();
+  private HttpPipelineFactory pipelineFact;
+  private int sslFileBufferSize;
+
+  private ApplicationId appId;
+  private FileSystem localFS;
+
+  /**
+   * Should the shuffle use posix_fadvise calls to manage the OS cache during
+   * sendfile
+   */
+  private boolean manageOsCache;
+  private int readaheadLength;
+  private ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
+
+
+  public static final String PULLSERVER_SERVICEID = "tajo.pullserver";
+
+  private static final Map<String,String> userRsrc =
+    new ConcurrentHashMap<String,String>();
+  private String userName;
+
+  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+    "tajo.pullserver.ssl.file.buffer.size";
+
+  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+  private static boolean STANDALONE = false;
+
+  static {
+    String standalone = System.getenv("TAJO_PULLSERVER_STANDALONE");
+    if (!StringUtils.isEmpty(standalone)) {
+      STANDALONE = standalone.equalsIgnoreCase("true");
+    }
+  }
+
+  @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo")
+  static class ShuffleMetrics implements ChannelFutureListener {
+    @Metric({"OutputBytes","PullServer output in bytes"})
+    MutableCounterLong shuffleOutputBytes;
+    @Metric({"Failed","# of failed shuffle outputs"})
+    MutableCounterInt shuffleOutputsFailed;
+    @Metric({"Succeeded","# of succeeded shuffle outputs"})
+    MutableCounterInt shuffleOutputsOK;
+    @Metric({"Connections","# of current shuffle connections"})
+    MutableGaugeInt shuffleConnections;
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if (future.isSuccess()) {
+        shuffleOutputsOK.incr();
+      } else {
+        shuffleOutputsFailed.incr();
+      }
+      shuffleConnections.decr();
+    }
+  }
+
+  final ShuffleMetrics metrics;
+
+  TajoPullServerService(MetricsSystem ms) {
+    super("httpshuffle");
+    metrics = ms.register(new ShuffleMetrics());
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public TajoPullServerService() {
+    this(DefaultMetricsSystem.instance());
+  }
+
+  /**
+   * Serialize the shuffle port into a ByteBuffer for use later on.
+   * @param port the port to be sent to the ApplciationMaster
+   * @return the serialized form of the port.
+   */
+  public static ByteBuffer serializeMetaData(int port) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer port_dob = new DataOutputBuffer();
+    port_dob.writeInt(port);
+    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+  }
+
+  /**
+   * A helper function to deserialize the metadata returned by PullServerAuxService.
+   * @param meta the metadata returned by the PullServerAuxService
+   * @return the port the PullServer Handler is listening on to serve shuffle data.
+   */
+  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+    //TODO this should be returning a class not just an int
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    return in.readInt();
+  }
+
+  public void initApp(String user, ApplicationId appId, ByteBuffer secret) {
+    // TODO these bytes should be versioned
+    // TODO: Once SHuffle is out of NM, this can use MR APIs
+    this.appId = appId;
+    this.userName = user;
+    userRsrc.put(appId.toString(), user);
+  }
+
+  public void stopApp(ApplicationId appId) {
+    userRsrc.remove(appId.toString());
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    try {
+      manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
+          DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
+
+      readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
+          DEFAULT_SHUFFLE_READAHEAD_BYTES);
+
+      int workerNum = conf.getInt("tajo.shuffle.rpc.server.worker-thread-num",
+          Runtime.getRuntime().availableProcessors() * 2);
+
+      selector = RpcChannelFactory.createServerChannelFactory("PullServerAuxService", workerNum);
+
+      localFS = new LocalFileSystem();
+
+      conf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname
+          , TajoConf.ConfVars.PULLSERVER_PORT.defaultIntVal);
+      super.init(conf);
+      LOG.info("Tajo PullServer initialized: readaheadLength=" + readaheadLength);
+    } catch (Throwable t) {
+      LOG.error(t);
+    }
+  }
+
+  // TODO change AbstractService to throw InterruptedException
+  @Override
+  public synchronized void serviceInit(Configuration conf) throws Exception {
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+
+    try {
+      pipelineFact = new HttpPipelineFactory(conf);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    bootstrap.setPipelineFactory(pipelineFact);
+
+    port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
+        ConfVars.PULLSERVER_PORT.defaultIntVal);
+    Channel ch = bootstrap.bind(new InetSocketAddress(port));
+
+    accepted.add(ch);
+    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    conf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port));
+    pipelineFact.PullServer.setPort(port);
+    LOG.info(getName() + " listening on port " + port);
+
+    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+
+
+    if (STANDALONE) {
+      File pullServerPortFile = getPullServerPortFile();
+      if (pullServerPortFile.exists()) {
+        pullServerPortFile.delete();
+      }
+      pullServerPortFile.getParentFile().mkdirs();
+      LOG.info("Write PullServerPort to " + pullServerPortFile);
+      FileOutputStream out = null;
+      try {
+        out = new FileOutputStream(pullServerPortFile);
+        out.write(("" + port).getBytes());
+      } catch (Exception e) {
+        LOG.fatal("PullServer exists cause can't write PullServer port to " + pullServerPortFile +
+            ", " + e.getMessage(), e);
+        System.exit(-1);
+      } finally {
+        IOUtils.closeStream(out);
+      }
+    }
+    super.serviceInit(conf);
+    LOG.info("TajoPullServerService started: port=" + port);
+  }
+
+  public static boolean isStandalone() {
+    return STANDALONE;
+  }
+
+  private static File getPullServerPortFile() {
+    String pullServerPortInfoFile = System.getenv("TAJO_PID_DIR");
+    if (StringUtils.isEmpty(pullServerPortInfoFile)) {
+      pullServerPortInfoFile = "/tmp";
+    }
+    return new File(pullServerPortInfoFile + "/pullserver.port");
+  }
+
+  // TODO change to get port from master or tajoConf
+  public static int readPullServerPort() {
+    FileInputStream in = null;
+    try {
+      File pullServerPortFile = getPullServerPortFile();
+
+      if (!pullServerPortFile.exists() || pullServerPortFile.isDirectory()) {
+        return -1;
+      }
+      in = new FileInputStream(pullServerPortFile);
+      byte[] buf = new byte[1024];
+      int readBytes = in.read(buf);
+      return Integer.parseInt(new String(buf, 0, readBytes));
+    } catch (IOException e) {
+      LOG.fatal(e.getMessage(), e);
+      return -1;
+    } finally {
+      IOUtils.closeStream(in);
+    }
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  @Override
+  public synchronized void stop() {
+    try {
+      accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+      ServerBootstrap bootstrap = new ServerBootstrap(selector);
+      bootstrap.releaseExternalResources();
+      pipelineFact.destroy();
+
+      localFS.close();
+    } catch (Throwable t) {
+      LOG.error(t);
+    } finally {
+      super.stop();
+    }
+  }
+
+  public synchronized ByteBuffer getMeta() {
+    try {
+      return serializeMetaData(port); 
+    } catch (IOException e) {
+      LOG.error("Error during getMeta", e);
+      // TODO add API to AuxiliaryServices to report failures
+      return null;
+    }
+  }
+
+  class HttpPipelineFactory implements ChannelPipelineFactory {
+
+    final PullServer PullServer;
+    private SSLFactory sslFactory;
+
+    public HttpPipelineFactory(Configuration conf) throws Exception {
+      PullServer = new PullServer(conf);
+      if (conf.getBoolean(ConfVars.SHUFFLE_SSL_ENABLED_KEY.varname,
+          ConfVars.SHUFFLE_SSL_ENABLED_KEY.defaultBoolVal)) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+        sslFactory.init();
+      }
+    }
+
+    public void destroy() {
+      if (sslFactory != null) {
+        sslFactory.destroy();
+      }
+    }
+
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+      ChannelPipeline pipeline = Channels.pipeline();
+      if (sslFactory != null) {
+        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+      }
+
+      int maxChunkSize = getConfig().getInt(ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE.varname,
+          ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE.defaultIntVal);
+      pipeline.addLast("codec", new HttpServerCodec(4096, 8192, maxChunkSize));
+      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+      pipeline.addLast("chunking", new ChunkedWriteHandler());
+      pipeline.addLast("shuffle", PullServer);
+      return pipeline;
+      // TODO factor security manager into pipeline
+      // TODO factor out encode/decode to permit binary shuffle
+      // TODO factor out decode of index to permit alt. models
+    }
+  }
+
+
+  Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>();
+
+  public void completeFileChunk(FileRegion filePart,
+                                   String requestUri,
+                                   long startTime) {
+    ProcessingStatus status = processingStatusMap.get(requestUri);
+    if (status != null) {
+      status.decrementRemainFiles(filePart, startTime);
+    }
+  }
+
+  class ProcessingStatus {
+    String requestUri;
+    int numFiles;
+    AtomicInteger remainFiles;
+    long startTime;
+    long makeFileListTime;
+    long minTime = Long.MAX_VALUE;
+    long maxTime;
+    int numSlowFile;
+
+    public ProcessingStatus(String requestUri) {
+      this.requestUri = requestUri;
+      this.startTime = System.currentTimeMillis();
+    }
+
+    public void setNumFiles(int numFiles) {
+      this.numFiles = numFiles;
+      this.remainFiles = new AtomicInteger(numFiles);
+    }
+    public void decrementRemainFiles(FileRegion filePart, long fileStartTime) {
+      synchronized(remainFiles) {
+        long fileSendTime = System.currentTimeMillis() - fileStartTime;
+        if (fileSendTime > 20 * 1000) {
+          LOG.info("PullServer send too long time: filePos=" + filePart.getPosition() + ", fileLen=" + filePart.getCount());
+          numSlowFile++;
+        }
+        if (fileSendTime > maxTime) {
+          maxTime = fileSendTime;
+        }
+        if (fileSendTime < minTime) {
+          minTime = fileSendTime;
+        }
+        int remain = remainFiles.decrementAndGet();
+        if (remain <= 0) {
+          processingStatusMap.remove(requestUri);
+          LOG.info("PullServer processing status: totalTime=" + (System.currentTimeMillis() - startTime) + " ms, " +
+              "makeFileListTime=" + makeFileListTime + " ms, minTime=" + minTime + " ms, maxTime=" + maxTime + " ms, " +
+              "numFiles=" + numFiles + ", numSlowFile=" + numSlowFile);
+        }
+      }
+    }
+  }
+
+  class PullServer extends SimpleChannelUpstreamHandler {
+
+    private final Configuration conf;
+//    private final IndexCache indexCache;
+    private final LocalDirAllocator lDirAlloc =
+      new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
+    private int port;
+
+    public PullServer(Configuration conf) throws IOException {
+      this.conf = conf;
+//      indexCache = new IndexCache(new JobConf(conf));
+      this.port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
+          ConfVars.PULLSERVER_PORT.defaultIntVal);
+
+      // init local temporal dir
+      lDirAlloc.getAllLocalPathsToRead(".", conf);
+    }
+    
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    private List<String> splitMaps(List<String> mapq) {
+      if (null == mapq) {
+        return null;
+      }
+      final List<String> ret = new ArrayList<String>();
+      for (String s : mapq) {
+        Collections.addAll(ret, s.split(","));
+      }
+      return ret;
+    }
+
+    @Override
+    public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt)
+        throws Exception {
+
+      accepted.add(evt.getChannel());
+      LOG.info(String.format("Current number of shuffle connections (%d)", accepted.size()));
+      super.channelOpen(ctx, evt);
+
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+        throws Exception {
+
+      HttpRequest request = (HttpRequest) e.getMessage();
+      if (request.getMethod() != GET) {
+        sendError(ctx, METHOD_NOT_ALLOWED);
+        return;
+      }
+
+      ProcessingStatus processingStatus = new ProcessingStatus(request.getUri().toString());
+      processingStatusMap.put(request.getUri().toString(), processingStatus);
+      // Parsing the URL into key-values
+      final Map<String, List<String>> params =
+          new QueryStringDecoder(request.getUri()).getParameters();
+      final List<String> types = params.get("type");
+      final List<String> qids = params.get("qid");
+      final List<String> taskIdList = params.get("ta");
+      final List<String> subQueryIds = params.get("sid");
+      final List<String> partIds = params.get("p");
+      final List<String> offsetList = params.get("offset");
+      final List<String> lengthList = params.get("length");
+
+      if (types == null || subQueryIds == null || qids == null || partIds == null) {
+        sendError(ctx, "Required queryId, type, subquery Id, and part id",
+            BAD_REQUEST);
+        return;
+      }
+
+      if (qids.size() != 1 && types.size() != 1 || subQueryIds.size() != 1) {
+        sendError(ctx, "Required qids, type, taskIds, subquery Id, and part id",
+            BAD_REQUEST);
+        return;
+      }
+
+      String partId = partIds.get(0);
+      String queryId = qids.get(0);
+      String shuffleType = types.get(0);
+      String sid = subQueryIds.get(0);
+
+      long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L;
+      long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L;
+
+      if (!shuffleType.equals("h") && !shuffleType.equals("s") && taskIdList == null) {
+        sendError(ctx, "Required taskIds", BAD_REQUEST);
+      }
+
+      List<String> taskIds = splitMaps(taskIdList);
+
+      String queryBaseDir = queryId.toString() + "/output";
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("PullServer request param: shuffleType=" + shuffleType +
+            ", sid=" + sid + ", partId=" + partId + ", taskIds=" + taskIdList);
+
+        // the working dir of tajo worker for each query
+        LOG.debug("PullServer baseDir: " + conf.get(ConfVars.WORKER_TEMPORAL_DIR.varname) + "/" + queryBaseDir);
+      }
+
+      final List<FileChunk> chunks = Lists.newArrayList();
+
+      // if a subquery requires a range shuffle
+      if (shuffleType.equals("r")) {
+        String ta = taskIds.get(0);
+        if(!lDirAlloc.ifExists(queryBaseDir + "/" + sid + "/" + ta + "/output/", conf)){
+          LOG.warn(e);
+          sendError(ctx, NO_CONTENT);
+          return;
+        }
+        Path path = localFS.makeQualified(
+            lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" + ta + "/output/", conf));
+        String startKey = params.get("start").get(0);
+        String endKey = params.get("end").get(0);
+        boolean last = params.get("final") != null;
+
+        FileChunk chunk;
+        try {
+          chunk = getFileCunks(path, startKey, endKey, last);
+        } catch (Throwable t) {
+          LOG.error("ERROR Request: " + request.getUri(), t);
+          sendError(ctx, "Cannot get file chunks to be sent", BAD_REQUEST);
+          return;
+        }
+        if (chunk != null) {
+          chunks.add(chunk);
+        }
+
+        // if a subquery requires a hash 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 + "/" + sid + "/hash-shuffle/" + partParentId + "/" + partId;
+        if (!lDirAlloc.ifExists(partPath, conf)) {
+          LOG.warn("Partition shuffle file not exists: " + partPath);
+          sendError(ctx, NO_CONTENT);
+          return;
+        }
+
+        Path path = localFS.makeQualified(lDirAlloc.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()) {
+          String errorMessage = "Start pos[" + startPos + "] great than file length [" + file.length() + "]";
+          LOG.error(errorMessage);
+          sendError(ctx, errorMessage, BAD_REQUEST);
+          return;
+        }
+        LOG.info("RequestURL: " + request.getUri() + ", fileLen=" + file.length());
+        FileChunk chunk = new FileChunk(file, startPos, readLen);
+        chunks.add(chunk);
+      } else {
+        LOG.error("Unknown shuffle type: " + shuffleType);
+        sendError(ctx, "Unknown shuffle type:" + shuffleType, BAD_REQUEST);
+        return;
+      }
+
+      processingStatus.setNumFiles(chunks.size());
+      processingStatus.makeFileListTime = System.currentTimeMillis() - processingStatus.startTime;
+      // Write the content.
+      Channel ch = e.getChannel();
+      if (chunks.size() == 0) {
+        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
+        ch.write(response);
+        if (!isKeepAlive(request)) {
+          ch.close();
+        }
+      }  else {
+        FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
+        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+        long totalSize = 0;
+        for (FileChunk chunk : file) {
+          totalSize += chunk.length();
+        }
+        setContentLength(response, totalSize);
+
+        // Write the initial line and the header.
+        ch.write(response);
+
+        ChannelFuture writeFuture = null;
+
+        for (FileChunk chunk : file) {
+          writeFuture = sendFile(ctx, ch, chunk, request.getUri().toString());
+          if (writeFuture == null) {
+            sendError(ctx, NOT_FOUND);
+            return;
+          }
+        }
+
+        // Decide whether to close the connection or not.
+        if (!isKeepAlive(request)) {
+          // Close the connection when the whole content is written out.
+          writeFuture.addListener(ChannelFutureListener.CLOSE);
+        }
+      }
+    }
+
+    private ChannelFuture sendFile(ChannelHandlerContext ctx,
+                                   Channel ch,
+                                   FileChunk file,
+                                   String requestUri) throws IOException {
+      long startTime = System.currentTimeMillis();
+      RandomAccessFile spill = null;
+      ChannelFuture writeFuture;
+      try {
+        spill = new RandomAccessFile(file.getFile(), "r");
+        if (ch.getPipeline().get(SslHandler.class) == null) {
+          final FadvisedFileRegion filePart = new FadvisedFileRegion(spill,
+              file.startOffset(), file.length(), manageOsCache, readaheadLength,
+              readaheadPool, file.getFile().getAbsolutePath());
+          writeFuture = ch.write(filePart);
+          writeFuture.addListener(new FileCloseListener(filePart, requestUri, startTime, TajoPullServerService.this));
+        } else {
+          // HTTPS cannot be done with zero copy.
+          final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
+              file.startOffset(), file.length(), sslFileBufferSize,
+              manageOsCache, readaheadLength, readaheadPool,
+              file.getFile().getAbsolutePath());
+          writeFuture = ch.write(chunk);
+        }
+      } catch (FileNotFoundException e) {
+        LOG.info(file.getFile() + " not found");
+        return null;
+      } catch (Throwable e) {
+        if (spill != null) {
+          //should close a opening file
+          spill.close();
+        }
+        return null;
+      }
+      metrics.shuffleConnections.incr();
+      metrics.shuffleOutputBytes.incr(file.length()); // optimistic
+      return writeFuture;
+    }
+
+    private void sendError(ChannelHandlerContext ctx,
+        HttpResponseStatus status) {
+      sendError(ctx, "", status);
+    }
+
+    private void sendError(ChannelHandlerContext ctx, String message,
+        HttpResponseStatus status) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+      response.setContent(
+        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+
+      // Close the connection as soon as the error message is sent.
+      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+        throws Exception {
+      LOG.error(e.getCause().getMessage(), e.getCause());
+      //if channel.close() is not called, never closed files in this request
+      if (ctx.getChannel().isConnected()){
+        ctx.getChannel().close();
+      }
+    }
+  }
+
+  public static FileChunk getFileCunks(Path outDir,
+                                      String startKey,
+                                      String endKey,
+                                      boolean last) throws IOException {
+    BSTIndex index = new BSTIndex(new TajoConf());
+    BSTIndex.BSTIndexReader idxReader =
+        index.getIndexReader(new Path(outDir, "index"));
+    idxReader.open();
+    Schema keySchema = idxReader.getKeySchema();
+    TupleComparator comparator = idxReader.getComparator();
+
+    LOG.info("BSTIndex is loaded from disk (" + idxReader.getFirstKey() + ", "
+        + idxReader.getLastKey());
+
+    File data = new File(URI.create(outDir.toUri() + "/output"));
+    byte [] startBytes = Base64.decodeBase64(startKey);
+    byte [] endBytes = Base64.decodeBase64(endKey);
+
+    RowStoreDecoder decoder = RowStoreUtil.createDecoder(keySchema);
+    Tuple start;
+    Tuple end;
+    try {
+      start = decoder.toTuple(startBytes);
+    } catch (Throwable t) {
+      throw new IllegalArgumentException("StartKey: " + startKey
+          + ", decoded byte size: " + startBytes.length, t);
+    }
+
+    try {
+      end = decoder.toTuple(endBytes);
+    } catch (Throwable t) {
+      throw new IllegalArgumentException("EndKey: " + endKey
+          + ", decoded byte size: " + endBytes.length, t);
+    }
+
+    LOG.info("GET Request for " + data.getAbsolutePath() + " (start="+start+", end="+ end +
+        (last ? ", last=true" : "") + ")");
+
+    if (idxReader.getFirstKey() == null && idxReader.getLastKey() == null) { // if # of rows is zero
+      LOG.info("There is no contents");
+      return null;
+    }
+
+    if (comparator.compare(end, idxReader.getFirstKey()) < 0 ||
+        comparator.compare(idxReader.getLastKey(), start) < 0) {
+      LOG.warn("Out of Scope (indexed data [" + idxReader.getFirstKey() + ", " + idxReader.getLastKey() +
+          "], but request start:" + start + ", end: " + end);
+      return null;
+    }
+
+    long startOffset;
+    long endOffset;
+    try {
+      startOffset = idxReader.find(start);
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + "[" + start + ", " + end +")" + ", idx min: "
+          + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+    try {
+      endOffset = idxReader.find(end);
+      if (endOffset == -1) {
+        endOffset = idxReader.find(end, true);
+      }
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + "[" + start + ", " + end +")" + ", idx min: "
+          + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+
+    // if startOffset == -1 then case 2-1 or case 3
+    if (startOffset == -1) { // this is a hack
+      // if case 2-1 or case 3
+      try {
+        startOffset = idxReader.find(start, true);
+      } catch (IOException ioe) {
+        LOG.error("State Dump (the requested range: "
+            + "[" + start + ", " + end +")" + ", idx min: "
+            + idxReader.getFirstKey() + ", idx max: "
+            + idxReader.getLastKey());
+        throw ioe;
+      }
+    }
+
+    if (startOffset == -1) {
+      throw new IllegalStateException("startOffset " + startOffset + " is negative \n" +
+          "State Dump (the requested range: "
+          + "[" + start + ", " + end +")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+    }
+
+    // if greater than indexed values
+    if (last || (endOffset == -1
+        && comparator.compare(idxReader.getLastKey(), end) < 0)) {
+      endOffset = data.length();
+    }
+
+    idxReader.close();
+
+    FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset);
+    LOG.info("Retrieve File Chunk: " + chunk);
+    return chunk;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
new file mode 100644
index 0000000..67e7423
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver.retriever;
+
+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.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.QueryUnitId;
+import org.apache.tajo.pullserver.FileAccessForbiddenException;
+import org.apache.tajo.util.TajoIdUtils;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class AdvancedDataRetriever implements DataRetriever {
+  private final Log LOG = LogFactory.getLog(AdvancedDataRetriever.class);
+  private final Map<String, RetrieverHandler> handlerMap = Maps.newConcurrentMap();
+
+  public AdvancedDataRetriever() {
+  }
+  
+  public void register(String taskAttemptId, RetrieverHandler handler) {
+    synchronized (handlerMap) {
+      if (!handlerMap.containsKey(taskAttemptId)) {
+        handlerMap.put(taskAttemptId, handler);
+      }
+    } 
+  }
+  
+  public void unregister(String taskAttemptId) {
+    synchronized (handlerMap) {
+      if (handlerMap.containsKey(taskAttemptId)) {
+        handlerMap.remove(taskAttemptId);
+      }
+    }
+  }
+
+  @Override
+  public FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
+      throws IOException {
+
+    final Map<String, List<String>> params =
+      new QueryStringDecoder(request.getUri()).getParameters();
+
+    if (!params.containsKey("qid")) {
+      throw new FileNotFoundException("No such qid: " + params.containsKey("qid"));
+    }
+
+    if (params.containsKey("sid")) {
+      List<FileChunk> chunks = Lists.newArrayList();
+      List<String> queryUnidIds = splitMaps(params.get("qid"));
+      for (String eachQueryUnitId : queryUnidIds) {
+        String[] queryUnitIdSeqTokens = eachQueryUnitId.split("_");
+        ExecutionBlockId ebId = TajoIdUtils.createExecutionBlockId(params.get("sid").get(0));
+        QueryUnitId quid = new QueryUnitId(ebId, Integer.parseInt(queryUnitIdSeqTokens[0]));
+
+        QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid, Integer.parseInt(queryUnitIdSeqTokens[1]));
+
+        RetrieverHandler handler = handlerMap.get(attemptId.toString());
+        FileChunk chunk = handler.get(params);
+        chunks.add(chunk);
+      }
+      return chunks.toArray(new FileChunk[chunks.size()]);
+    } else {
+      RetrieverHandler handler = handlerMap.get(params.get("qid").get(0));
+      FileChunk chunk = handler.get(params);
+      if (chunk == null) {
+        if (params.containsKey("qid")) { // if there is no content corresponding to the query
+          return null;
+        } else { // if there is no
+          throw new FileNotFoundException("No such a file corresponding to " + params.get("qid"));
+        }
+      }
+
+      File file = chunk.getFile();
+      if (file.isHidden() || !file.exists()) {
+        throw new FileNotFoundException("No such file: " + file.getAbsolutePath());
+      }
+      if (!file.isFile()) {
+        throw new FileAccessForbiddenException(file.getAbsolutePath() + " is not file");
+      }
+
+      return new FileChunk[] {chunk};
+    }
+  }
+
+  private List<String> splitMaps(List<String> qids) {
+    if (null == qids) {
+      LOG.error("QueryUnitId is EMPTY");
+      return null;
+    }
+
+    final List<String> ret = new ArrayList<String>();
+    for (String qid : qids) {
+      Collections.addAll(ret, qid.split(","));
+    }
+    return ret;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.java
new file mode 100644
index 0000000..8f55f7b
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver.retriever;
+
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+
+import java.io.IOException;
+
+public interface DataRetriever {
+  FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java
new file mode 100644
index 0000000..dc63929
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver.retriever;
+
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.apache.tajo.pullserver.FileAccessForbiddenException;
+import org.apache.tajo.pullserver.HttpDataServerHandler;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class DirectoryRetriever implements DataRetriever {
+  public String baseDir;
+  
+  public DirectoryRetriever(String baseDir) {
+    this.baseDir = baseDir;
+  }
+
+  @Override
+  public FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
+      throws IOException {
+    final String path = HttpDataServerHandler.sanitizeUri(request.getUri());
+    if (path == null) {
+      throw new IllegalArgumentException("Wrong path: " +path);
+    }
+
+    File file = new File(baseDir, path);
+    if (file.isHidden() || !file.exists()) {
+      throw new FileNotFoundException("No such file: " + baseDir + "/" + path);
+    }
+    if (!file.isFile()) {
+      throw new FileAccessForbiddenException("No such file: "
+          + baseDir + "/" + path); 
+    }
+    
+    return new FileChunk[] {new FileChunk(file, 0, file.length())};
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java
new file mode 100644
index 0000000..67cff21
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver.retriever;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+
+public class FileChunk {
+  private final File file;
+  private final long startOffset;
+  private long length;
+
+  /**
+   * TRUE if this.file is created by getting data from a remote host (e.g., by HttpRequest). FALSE otherwise.
+   */
+  private boolean fromRemote;
+
+  /**
+   * ExecutionBlockId
+   */
+  private String ebId;
+
+  public FileChunk(File file, long startOffset, long length) throws FileNotFoundException {
+    this.file = file;
+    this.startOffset = startOffset;
+    this.length = length;
+  }
+
+  public File getFile() {
+    return this.file;
+  }
+
+  public long startOffset() {
+    return this.startOffset;
+  }
+
+  public long length() {
+    return this.length;
+  }
+
+  public void setLength(long newLength) {
+    this.length = newLength;
+  }
+
+  public boolean fromRemote() {
+    return this.fromRemote;
+  }
+
+  public void setFromRemote(boolean newVal) {
+    this.fromRemote = newVal;
+  }
+
+  public String getEbId() {
+    return this.ebId;
+  }
+
+  public void setEbId(String newVal) {
+    this.ebId = newVal;
+  }
+
+  public String toString() {
+    return " (start=" + startOffset() + ", length=" + length + ", fromRemote=" + fromRemote + ", ebId=" + ebId + ") "
+	+ file.getAbsolutePath();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java
new file mode 100644
index 0000000..5567c0d
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver.retriever;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+public interface RetrieverHandler {
+  /**
+   *
+   * @param kvs url-decoded key/value pairs
+   * @return a desired part of a file
+   * @throws java.io.IOException
+   */
+  public FileChunk get(Map<String, List<String>> kvs) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/pom.xml b/tajo-yarn-pullserver/pom.xml
deleted file mode 100644
index a7644a1..0000000
--- a/tajo-yarn-pullserver/pom.xml
+++ /dev/null
@@ -1,146 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
-  -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <parent>
-    <artifactId>tajo-project</artifactId>
-    <groupId>org.apache.tajo</groupId>
-    <version>0.9.1-SNAPSHOT</version>
-    <relativePath>../tajo-project</relativePath>
-  </parent>
-  <modelVersion>4.0.0</modelVersion>
-  <name>Tajo Core PullServer</name>
-  <artifactId>tajo-yarn-pullserver</artifactId>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>verify</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-catalog-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-nodemanager</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-shuffle</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>commons-el</groupId>
-          <artifactId>commons-el</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-runtime</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-compiler</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jsp-2.1-jetty</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-  </dependencies>
-
-  <profiles>
-    <profile>
-      <id>docs</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-javadoc-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- build javadoc jars per jar for publishing to maven -->
-                <id>module-javadocs</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>jar</goal>
-                </goals>
-                <configuration>
-                  <destDir>${project.build.directory}</destDir>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
-  <reporting>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-project-info-reports-plugin</artifactId>
-        <version>2.4</version>
-        <configuration>
-          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
-        </configuration>
-      </plugin>
-    </plugins>
-  </reporting>
-
-</project>

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java
deleted file mode 100644
index b0b8d18..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java
+++ /dev/null
@@ -1,81 +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.pullserver;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.ReadaheadPool;
-import org.apache.hadoop.io.nativeio.NativeIO;
-import org.jboss.netty.handler.stream.ChunkedFile;
-
-import java.io.FileDescriptor;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-
-public class FadvisedChunkedFile extends ChunkedFile {
-
-  private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
-
-  private final boolean manageOsCache;
-  private final int readaheadLength;
-  private final ReadaheadPool readaheadPool;
-  private final FileDescriptor fd;
-  private final String identifier;
-
-  private ReadaheadPool.ReadaheadRequest readaheadRequest;
-
-  public FadvisedChunkedFile(RandomAccessFile file, long position, long count,
-                             int chunkSize, boolean manageOsCache, int readaheadLength,
-                             ReadaheadPool readaheadPool, String identifier) throws IOException {
-    super(file, position, count, chunkSize);
-    this.manageOsCache = manageOsCache;
-    this.readaheadLength = readaheadLength;
-    this.readaheadPool = readaheadPool;
-    this.fd = file.getFD();
-    this.identifier = identifier;
-  }
-
-  @Override
-  public Object nextChunk() throws Exception {
-    if (PullServerUtil.isNativeIOPossible() && manageOsCache && readaheadPool != null) {
-      readaheadRequest = readaheadPool
-          .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength,
-              getEndOffset(), readaheadRequest);
-    }
-    return super.nextChunk();
-  }
-
-  @Override
-  public void close() throws Exception {
-    if (readaheadRequest != null) {
-      readaheadRequest.cancel();
-    }
-    if (PullServerUtil.isNativeIOPossible() && manageOsCache && getEndOffset() - getStartOffset() > 0) {
-      try {
-        PullServerUtil.posixFadviseIfPossible(identifier,
-            fd,
-            getStartOffset(), getEndOffset() - getStartOffset(),
-            NativeIO.POSIX.POSIX_FADV_DONTNEED);
-      } catch (Throwable t) {
-        LOG.warn("Failed to manage OS cache for " + identifier, t);
-      }
-    }
-    super.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java
deleted file mode 100644
index 18cf4b6..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java
+++ /dev/null
@@ -1,170 +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.pullserver;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.ReadaheadPool;
-import org.apache.hadoop.io.nativeio.NativeIO;
-import org.jboss.netty.channel.DefaultFileRegion;
-
-import java.io.FileDescriptor;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.WritableByteChannel;
-
-public class FadvisedFileRegion extends DefaultFileRegion {
-
-  private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class);
-
-  private final boolean manageOsCache;
-  private final int readaheadLength;
-  private final ReadaheadPool readaheadPool;
-  private final FileDescriptor fd;
-  private final String identifier;
-  private final long count;
-  private final long position;
-  private final int shuffleBufferSize;
-  private final boolean shuffleTransferToAllowed;
-  private final FileChannel fileChannel;
-
-  private ReadaheadPool.ReadaheadRequest readaheadRequest;
-  public static final int DEFAULT_SHUFFLE_BUFFER_SIZE = 128 * 1024;
-
-  public FadvisedFileRegion(RandomAccessFile file, long position, long count,
-                            boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool,
-                            String identifier) throws IOException {
-    this(file, position, count, manageOsCache, readaheadLength, readaheadPool,
-        identifier, DEFAULT_SHUFFLE_BUFFER_SIZE, true);
-  }
-
-  public FadvisedFileRegion(RandomAccessFile file, long position, long count,
-                            boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool,
-                            String identifier, int shuffleBufferSize,
-                            boolean shuffleTransferToAllowed) throws IOException {
-    super(file.getChannel(), position, count);
-    this.manageOsCache = manageOsCache;
-    this.readaheadLength = readaheadLength;
-    this.readaheadPool = readaheadPool;
-    this.fd = file.getFD();
-    this.identifier = identifier;
-    this.fileChannel = file.getChannel();
-    this.count = count;
-    this.position = position;
-    this.shuffleBufferSize = shuffleBufferSize;
-    this.shuffleTransferToAllowed = shuffleTransferToAllowed;
-  }
-
-  @Override
-  public long transferTo(WritableByteChannel target, long position)
-      throws IOException {
-    if (PullServerUtil.isNativeIOPossible() && manageOsCache && readaheadPool != null) {
-      readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
-          getPosition() + position, readaheadLength,
-          getPosition() + getCount(), readaheadRequest);
-    }
-
-    if(this.shuffleTransferToAllowed) {
-      return super.transferTo(target, position);
-    } else {
-      return customShuffleTransfer(target, position);
-    }
-  }
-
-  /**
-   * This method transfers data using local buffer. It transfers data from
-   * a disk to a local buffer in memory, and then it transfers data from the
-   * buffer to the target. This is used only if transferTo is disallowed in
-   * the configuration file. super.TransferTo does not perform well on Windows
-   * due to a small IO request generated. customShuffleTransfer can control
-   * the size of the IO requests by changing the size of the intermediate
-   * buffer.
-   */
-  @VisibleForTesting
-  long customShuffleTransfer(WritableByteChannel target, long position)
-      throws IOException {
-    long actualCount = this.count - position;
-    if (actualCount < 0 || position < 0) {
-      throw new IllegalArgumentException(
-          "position out of range: " + position +
-              " (expected: 0 - " + (this.count - 1) + ')');
-    }
-    if (actualCount == 0) {
-      return 0L;
-    }
-
-    long trans = actualCount;
-    int readSize;
-    ByteBuffer byteBuffer = ByteBuffer.allocate(this.shuffleBufferSize);
-
-    while(trans > 0L &&
-        (readSize = fileChannel.read(byteBuffer, this.position+position)) > 0) {
-      //adjust counters and buffer limit
-      if(readSize < trans) {
-        trans -= readSize;
-        position += readSize;
-        byteBuffer.flip();
-      } else {
-        //We can read more than we need if the actualCount is not multiple
-        //of the byteBuffer size and file is big enough. In that case we cannot
-        //use flip method but we need to set buffer limit manually to trans.
-        byteBuffer.limit((int)trans);
-        byteBuffer.position(0);
-        position += trans;
-        trans = 0;
-      }
-
-      //write data to the target
-      while(byteBuffer.hasRemaining()) {
-        target.write(byteBuffer);
-      }
-
-      byteBuffer.clear();
-    }
-
-    return actualCount - trans;
-  }
-
-
-  @Override
-  public void releaseExternalResources() {
-    if (readaheadRequest != null) {
-      readaheadRequest.cancel();
-    }
-    super.releaseExternalResources();
-  }
-
-  /**
-   * Call when the transfer completes successfully so we can advise the OS that
-   * we don't need the region to be cached anymore.
-   */
-  public void transferSuccessful() {
-    if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0) {
-      try {
-        PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(),
-            NativeIO.POSIX.POSIX_FADV_DONTNEED);
-      } catch (Throwable t) {
-        LOG.warn("Failed to manage OS cache for " + identifier, t);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java
deleted file mode 100644
index c703f6f..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java
+++ /dev/null
@@ -1,40 +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.pullserver;
-
-import java.io.IOException;
-
-public class FileAccessForbiddenException extends IOException {
-  private static final long serialVersionUID = -3383272565826389213L;
-
-  public FileAccessForbiddenException() {
-  }
-
-  public FileAccessForbiddenException(String message) {
-    super(message);
-  }
-
-  public FileAccessForbiddenException(Throwable cause) {
-    super(cause);
-  }
-
-  public FileAccessForbiddenException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java
deleted file mode 100644
index 236db89..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java
+++ /dev/null
@@ -1,53 +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.pullserver;
-
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-
-public class FileCloseListener implements ChannelFutureListener {
-
-  private FadvisedFileRegion filePart;
-  private String requestUri;
-  private TajoPullServerService pullServerService;
-  private long startTime;
-
-  public FileCloseListener(FadvisedFileRegion filePart,
-                           String requestUri,
-                           long startTime,
-                           TajoPullServerService pullServerService) {
-    this.filePart = filePart;
-    this.requestUri = requestUri;
-    this.pullServerService = pullServerService;
-    this.startTime = startTime;
-  }
-
-  // TODO error handling; distinguish IO/connection failures,
-  //      attribute to appropriate spill output
-  @Override
-  public void operationComplete(ChannelFuture future) {
-    if(future.isSuccess()){
-      filePart.transferSuccessful();
-    }
-    filePart.releaseExternalResources();
-    if (pullServerService != null) {
-      pullServerService.completeFileChunk(filePart, requestUri, startTime);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
deleted file mode 100644
index 31db15c..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
+++ /dev/null
@@ -1,245 +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.pullserver;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.pullserver.retriever.DataRetriever;
-import org.apache.tajo.pullserver.retriever.FileChunk;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.*;
-import org.jboss.netty.handler.codec.frame.TooLongFrameException;
-import org.jboss.netty.handler.codec.http.*;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.handler.stream.ChunkedFile;
-import org.jboss.netty.util.CharsetUtil;
-
-import java.io.*;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
-import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-
-public class HttpDataServerHandler extends SimpleChannelUpstreamHandler {
-  private final static Log LOG = LogFactory.getLog(HttpDataServerHandler.class);
-
-  Map<ExecutionBlockId, DataRetriever> retrievers =
-      new ConcurrentHashMap<ExecutionBlockId, DataRetriever>();
-  private String userName;
-  private String appId;
-
-  public HttpDataServerHandler(String userName, String appId) {
-    this.userName= userName;
-    this.appId = appId;
-  }
-
-  @Override
-  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
-      throws Exception {
-    HttpRequest request = (HttpRequest) e.getMessage();
-    if (request.getMethod() != GET) {
-      sendError(ctx, METHOD_NOT_ALLOWED);
-      return;
-    }
-
-    String base =
-        ContainerLocalizer.USERCACHE + "/" + userName + "/"
-            + ContainerLocalizer.APPCACHE + "/"
-            + appId + "/output" + "/";
-
-    final Map<String, List<String>> params =
-        new QueryStringDecoder(request.getUri()).getParameters();
-
-    List<FileChunk> chunks = Lists.newArrayList();
-    List<String> taskIds = splitMaps(params.get("ta"));
-    int sid = Integer.valueOf(params.get("sid").get(0));
-    int partitionId = Integer.valueOf(params.get("p").get(0));
-    for (String ta : taskIds) {
-
-      File file = new File(base + "/" + sid + "/" + ta + "/output/" + partitionId);
-      FileChunk chunk = new FileChunk(file, 0, file.length());
-      chunks.add(chunk);
-    }
-
-    FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
-//    try {
-//      file = retriever.handle(ctx, request);
-//    } catch (FileNotFoundException fnf) {
-//      LOG.error(fnf);
-//      sendError(ctx, NOT_FOUND);
-//      return;
-//    } catch (IllegalArgumentException iae) {
-//      LOG.error(iae);
-//      sendError(ctx, BAD_REQUEST);
-//      return;
-//    } catch (FileAccessForbiddenException fafe) {
-//      LOG.error(fafe);
-//      sendError(ctx, FORBIDDEN);
-//      return;
-//    } catch (IOException ioe) {
-//      LOG.error(ioe);
-//      sendError(ctx, INTERNAL_SERVER_ERROR);
-//      return;
-//    }
-
-    // Write the content.
-    Channel ch = e.getChannel();
-    if (file == null) {
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
-      ch.write(response);
-      if (!isKeepAlive(request)) {
-        ch.close();
-      }
-    }  else {
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-      long totalSize = 0;
-      for (FileChunk chunk : file) {
-        totalSize += chunk.length();
-      }
-      setContentLength(response, totalSize);
-
-      // Write the initial line and the header.
-      ch.write(response);
-
-      ChannelFuture writeFuture = null;
-
-      for (FileChunk chunk : file) {
-        writeFuture = sendFile(ctx, ch, chunk);
-        if (writeFuture == null) {
-          sendError(ctx, NOT_FOUND);
-          return;
-        }
-      }
-
-      // Decide whether to close the connection or not.
-      if (!isKeepAlive(request)) {
-        // Close the connection when the whole content is written out.
-        writeFuture.addListener(ChannelFutureListener.CLOSE);
-      }
-    }
-  }
-
-  private ChannelFuture sendFile(ChannelHandlerContext ctx,
-                                 Channel ch,
-                                 FileChunk file) throws IOException {
-    RandomAccessFile raf;
-    try {
-      raf = new RandomAccessFile(file.getFile(), "r");
-    } catch (FileNotFoundException fnfe) {
-      return null;
-    }
-
-    ChannelFuture writeFuture;
-    if (ch.getPipeline().get(SslHandler.class) != null) {
-      // Cannot use zero-copy with HTTPS.
-      writeFuture = ch.write(new ChunkedFile(raf, file.startOffset(),
-          file.length(), 8192));
-    } else {
-      // No encryption - use zero-copy.
-      final FileRegion region = new DefaultFileRegion(raf.getChannel(),
-          file.startOffset(), file.length());
-      writeFuture = ch.write(region);
-      writeFuture.addListener(new ChannelFutureListener() {
-        public void operationComplete(ChannelFuture future) {
-          region.releaseExternalResources();
-        }
-      });
-    }
-
-    return writeFuture;
-  }
-
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
-      throws Exception {
-    Channel ch = e.getChannel();
-    Throwable cause = e.getCause();
-    if (cause instanceof TooLongFrameException) {
-      sendError(ctx, BAD_REQUEST);
-      return;
-    }
-
-    cause.printStackTrace();
-    if (ch.isConnected()) {
-      sendError(ctx, INTERNAL_SERVER_ERROR);
-    }
-  }
-
-  public static String sanitizeUri(String uri) {
-    // Decode the path.
-    try {
-      uri = URLDecoder.decode(uri, "UTF-8");
-    } catch (UnsupportedEncodingException e) {
-      try {
-        uri = URLDecoder.decode(uri, "ISO-8859-1");
-      } catch (UnsupportedEncodingException e1) {
-        throw new Error();
-      }
-    }
-
-    // Convert file separators.
-    uri = uri.replace('/', File.separatorChar);
-
-    // Simplistic dumb security check.
-    // You will have to do something serious in the production environment.
-    if (uri.contains(File.separator + ".")
-        || uri.contains("." + File.separator) || uri.startsWith(".")
-        || uri.endsWith(".")) {
-      return null;
-    }
-
-    // Convert to absolute path.
-    return uri;
-  }
-
-  private void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
-    HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
-    response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
-    response.setContent(ChannelBuffers.copiedBuffer(
-        "Failure: " + status.toString() + "\r\n", CharsetUtil.UTF_8));
-
-    // Close the connection as soon as the error message is sent.
-    ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
-  }
-
-  private List<String> splitMaps(List<String> qids) {
-    if (null == qids) {
-      LOG.error("QueryUnitId is EMPTY");
-      return null;
-    }
-
-    final List<String> ret = new ArrayList<String>();
-    for (String qid : qids) {
-      Collections.addAll(ret, qid.split(","));
-    }
-    return ret;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java
deleted file mode 100644
index 4c8bd8b..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java
+++ /dev/null
@@ -1,56 +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.pullserver;
-
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.handler.codec.http.HttpContentCompressor;
-import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
-import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
-import org.jboss.netty.handler.stream.ChunkedWriteHandler;
-
-import static org.jboss.netty.channel.Channels.pipeline;
-
-public class HttpDataServerPipelineFactory implements ChannelPipelineFactory {
-  private String userName;
-  private String appId;
-  public HttpDataServerPipelineFactory(String userName, String appId) {
-    this.userName = userName;
-    this.appId = appId;
-  }
-
-  public ChannelPipeline getPipeline() throws Exception {
-    // Create a default pipeline implementation.
-    ChannelPipeline pipeline = pipeline();
-
-    // Uncomment the following line if you want HTTPS
-    // SSLEngine engine =
-    // SecureChatSslContextFactory.getServerContext().createSSLEngine();
-    // engine.setUseClientMode(false);
-    // pipeline.addLast("ssl", new SslHandler(engine));
-
-    pipeline.addLast("decoder", new HttpRequestDecoder());
-    //pipeline.addLast("aggregator", new HttpChunkAggregator(65536));
-    pipeline.addLast("encoder", new HttpResponseEncoder());
-    pipeline.addLast("chunkedWriter", new ChunkedWriteHandler());
-    pipeline.addLast("deflater", new HttpContentCompressor());
-    pipeline.addLast("handler", new HttpDataServerHandler(userName, appId));
-    return pipeline;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java
deleted file mode 100644
index 2cbb101..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java
+++ /dev/null
@@ -1,69 +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.pullserver;
-
-import com.google.common.collect.Maps;
-
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.net.URLEncoder;
-import java.util.Map;
-
-public class HttpUtil {
-  public static Map<String,String> getParams(URI uri) throws UnsupportedEncodingException {
-    return getParamsFromQuery(uri.getQuery());
-  }
-
-  /**
-   * It parses a query string into key/value pairs
-   *
-   * @param queryString decoded query string
-   * @return key/value pairs parsed from a given query string
-   * @throws java.io.UnsupportedEncodingException
-   */
-  public static Map<String, String> getParamsFromQuery(String queryString) throws UnsupportedEncodingException {
-    String [] queries = queryString.split("&");
-
-    Map<String,String> params = Maps.newHashMap();
-    String [] param;
-    for (String q : queries) {
-      param = q.split("=");
-      params.put(param[0], param[1]);
-    }
-
-    return params;
-  }
-
-  public static String buildQuery(Map<String,String> params) throws UnsupportedEncodingException {
-    StringBuilder sb = new StringBuilder();
-
-    boolean first = true;
-    for (Map.Entry<String,String> param : params.entrySet()) {
-      if (!first) {
-        sb.append("&");
-      }
-      sb.append(URLEncoder.encode(param.getKey(), "UTF-8")).
-          append("=").
-          append(URLEncoder.encode(param.getValue(), "UTF-8"));
-      first = false;
-    }
-
-    return sb.toString();
-  }
-}


[03/10] tajo git commit: TAJO-1234: Rearrange timezone in date/time types.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
----------------------------------------------------------------------
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 54c8722..bec8cd3 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
@@ -212,25 +212,25 @@ public class TestConditionalExpressions extends ExprTestBase {
   @Test
   public void testCoalesceTimestamp() throws Exception {
     testSimpleEval("select coalesce(null, timestamp '2014-01-01 00:00:00');",
-        new String[]{"2014-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-01 00:00:00"});
     testSimpleEval("select coalesce(null, null, timestamp '2014-01-01 00:00:00');",
-        new String[]{"2014-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-01 00:00:00"});
     testSimpleEval("select coalesce(timestamp '2014-01-01 00:00:00', null, timestamp '2014-01-02 00:00:00');",
-        new String[]{"2014-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-01 00:00:00"});
     testSimpleEval("select coalesce(null, timestamp '2014-01-01 00:00:00', timestamp '2014-02-01 00:00:00');",
-        new String[]{"2014-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-01 00:00:00"});
   }
 
   @Test
   public void testCoalesceTime() throws Exception {
     testSimpleEval("select coalesce(null, time '12:00:00');",
-        new String[]{"12:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"12:00:00"});
     testSimpleEval("select coalesce(null, null, time '12:00:00');",
-        new String[]{"12:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"12:00:00"});
     testSimpleEval("select coalesce(time '12:00:00', null, time '13:00:00');",
-        new String[]{"12:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"12:00:00"});
     testSimpleEval("select coalesce(null, time '12:00:00', time '13:00:00');",
-        new String[]{"12:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"12:00:00"});
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
----------------------------------------------------------------------
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 7cca13d..cb7856b 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
@@ -19,11 +19,13 @@
 package org.apache.tajo.engine.function;
 
 
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.TimestampDatum;
 import org.apache.tajo.engine.eval.ExprTestBase;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 import org.junit.Test;
@@ -34,6 +36,7 @@ import java.util.Date;
 import java.util.TimeZone;
 
 import static org.apache.tajo.common.TajoDataTypes.Type.*;
+import static org.junit.Assert.assertEquals;
 
 public class TestDateTimeFunctions extends ExprTestBase {
   @Test
@@ -43,58 +46,58 @@ public class TestDateTimeFunctions extends ExprTestBase {
 
     // (expectedTimestamp / 1000) means the translation from millis seconds to unix timestamp
     String q1 = String.format("select to_timestamp(%d);", (expectedTimestamp / 1000));
-    testSimpleEval(q1, new String[]{expected.toString(TajoConf.getCurrentTimeZone(), true)});
+    testSimpleEval(q1, new String[]{expected.toString()});
 
     testSimpleEval("select to_timestamp('1997-12-30 11:40:50.345', 'YYYY-MM-DD HH24:MI:SS.MS');",
-        new String[]{"1997-12-30 11:40:50.345" + getUserTimeZoneDisplay()});
+        new String[]{"1997-12-30 11:40:50.345"});
     testSimpleEval("select to_timestamp('1997-12-30 11:40:50.345 PM', 'YYYY-MM-DD HH24:MI:SS.MS PM');",
-        new String[]{"1997-12-30 23:40:50.345" + getUserTimeZoneDisplay()});
+        new String[]{"1997-12-30 23:40:50.345"});
     testSimpleEval("select to_timestamp('0097/Feb/16 --> 08:14:30', 'YYYY/Mon/DD --> HH:MI:SS');",
-        new String[]{"0097-02-16 08:14:30" + getUserTimeZoneDisplay()});
+        new String[]{"0097-02-16 08:14:30"});
     testSimpleEval("select to_timestamp('97/2/16 8:14:30', 'FMYYYY/FMMM/FMDD FMHH:FMMI:FMSS');",
-        new String[]{"0097-02-16 08:14:30" + getUserTimeZoneDisplay()});
+        new String[]{"0097-02-16 08:14:30"});
     testSimpleEval("select to_timestamp('1985 September 12', 'YYYY FMMonth DD');",
-        new String[]{"1985-09-12 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"1985-09-12 00:00:00"});
     testSimpleEval("select to_timestamp('1,582nd VIII 21', 'Y,YYYth FMRM DD');",
-        new String[]{"1582-08-21 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"1582-08-21 00:00:00"});
     testSimpleEval("select to_timestamp('05121445482000', 'MMDDHH24MISSYYYY');",
-        new String[]{"2000-05-12 14:45:48" + getUserTimeZoneDisplay()});
+        new String[]{"2000-05-12 14:45:48"});
     testSimpleEval("select to_timestamp('2000January09Sunday', 'YYYYFMMonthDDFMDay');",
-        new String[]{"2000-01-09 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2000-01-09 00:00:00"});
     testSimpleEval("select to_timestamp('97/Feb/16', 'YY/Mon/DD');",
-        new String[]{"1997-02-16 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"1997-02-16 00:00:00"});
     testSimpleEval("select to_timestamp('19971116', 'YYYYMMDD');",
-        new String[]{"1997-11-16 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"1997-11-16 00:00:00"});
     testSimpleEval("select to_timestamp('20000-1116', 'YYYY-MMDD');",
-        new String[]{"20000-11-16 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"20000-11-16 00:00:00"});
     testSimpleEval("select to_timestamp('9-1116', 'Y-MMDD');",
-        new String[]{"2009-11-16 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2009-11-16 00:00:00"});
     testSimpleEval("select to_timestamp('95-1116', 'YY-MMDD');",
-        new String[]{"1995-11-16 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"1995-11-16 00:00:00"});
     testSimpleEval("select to_timestamp('995-1116', 'YYY-MMDD');",
-        new String[]{"1995-11-16 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"1995-11-16 00:00:00"});
     testSimpleEval("select to_timestamp('2005426', 'YYYYWWD');",
-        new String[]{"2005-10-15 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2005-10-15 00:00:00"});
     testSimpleEval("select to_timestamp('2005300', 'YYYYDDD');",
-        new String[]{"2005-10-27 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2005-10-27 00:00:00"});
     testSimpleEval("select to_timestamp('2005527', 'IYYYIWID');",
-        new String[]{"2006-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2006-01-01 00:00:00"});
     testSimpleEval("select to_timestamp('005527', 'IYYIWID');",
-        new String[]{"2006-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2006-01-01 00:00:00"});
     testSimpleEval("select to_timestamp('05527', 'IYIWID');",
-        new String[]{"2006-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2006-01-01 00:00:00"});
     testSimpleEval("select to_timestamp('5527', 'IIWID');",
-        new String[]{"2006-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2006-01-01 00:00:00"});
     testSimpleEval("select to_timestamp('2005364', 'IYYYIDDD');",
-        new String[]{"2006-01-01 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2006-01-01 00:00:00"});
     testSimpleEval("select to_timestamp('20050302', 'YYYYMMDD');",
-        new String[]{"2005-03-02 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2005-03-02 00:00:00"});
     testSimpleEval("select to_timestamp('2005 03 02', 'YYYYMMDD');",
-        new String[]{"2005-03-02 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2005-03-02 00:00:00"});
     testSimpleEval("select to_timestamp(' 2005 03 02', 'YYYYMMDD');",
-        new String[]{"2005-03-02 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2005-03-02 00:00:00"});
     testSimpleEval("select to_timestamp('  20050302', 'YYYYMMDD');",
-        new String[]{"2005-03-02 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2005-03-02 00:00:00"});
   }
 
   @Test
@@ -115,10 +118,21 @@ public class TestDateTimeFunctions extends ExprTestBase {
 
   @Test
   public void testExtract() throws IOException {
+    TimeZone GMT = TimeZone.getTimeZone("GMT");
+    TimeZone PST = TimeZone.getTimeZone("PST");
+
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TIMESTAMP);
     testEval(schema2, "table1",
-        "1970-01-17 10:09:37" + getUserTimeZoneDisplay(),
+        "1970-01-17 10:09:37",
+        "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, "table1",
+        "1970-01-17 10:09:37" + getUserTimeZoneDisplay(GMT),
+        "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, "table1",
+        "1970-01-17 10:09:37" + getUserTimeZoneDisplay(PST),
         "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;",
         new String[]{"1970.0", "1.0", "17.0"});
 
@@ -126,9 +140,17 @@ public class TestDateTimeFunctions extends ExprTestBase {
     Schema schema3 = new Schema();
     schema3.addColumn("col1", TIME);
     testEval(schema3, "table1",
-        "10:09:37.5" + getUserTimeZoneDisplay(),
+        "10:09:37.5",
         "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, "table1",
+        "10:09:37.5" + getUserTimeZoneDisplay(GMT),
+        "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, "table1",
+        "10:09:37.5" + getUserTimeZoneDisplay(PST),
+        "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);
@@ -212,18 +234,36 @@ public class TestDateTimeFunctions extends ExprTestBase {
 
   @Test
   public void testDatePart() throws IOException {
+    TimeZone GMT = TimeZone.getTimeZone("GMT");
+    TimeZone PST = TimeZone.getTimeZone("PST");
+
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TIMESTAMP);
+
+    testEval(schema2, "table1",
+        "1970-01-17 22:09:37",
+        "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, "table1",
-        "1970-01-17 22:09:37" + getUserTimeZoneDisplay(),
+        "1970-01-17 22:09:37" + getUserTimeZoneDisplay(GMT),
         "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, "table1",
+        "1970-01-17 22:09:37" + getUserTimeZoneDisplay(PST),
+        "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, "table1", "10:09:37.5" + getUserTimeZoneDisplay(),
+    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, "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, "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);
@@ -318,7 +358,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   public void testToDate() throws IOException {
     testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD')", new String[]{"2014-01-04"});
     testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD') + interval '1 day'",
-        new String[]{"2014-01-05 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-05 00:00:00"});
 
     testSimpleEval("SELECT to_date('201404', 'yyyymm');", new String[]{"2014-04-01"});
   }
@@ -326,106 +366,110 @@ public class TestDateTimeFunctions extends ExprTestBase {
   @Test
   public void testAddMonths() throws Exception {
     testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT2);",
-        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-02-17 00:00:00"});
     testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT4);",
-        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-02-17 00:00:00"});
     testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT8);",
-        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-02-17 00:00:00"});
 
     testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT2);",
-        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2014-02-17 12:10:20"});
     testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT4);",
-        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2014-02-17 12:10:20"});
     testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT8);",
-        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2014-02-17 12:10:20"});
 
     testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT2);",
-        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-05 00:00:00"});
     testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT4);",
-        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-05 00:00:00"});
     testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT8);",
-        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-05 00:00:00"});
 
     testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT2);",
-        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-05 12:10:20"});
     testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT4);",
-        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-05 12:10:20"});
     testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT8);",
-        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-05 12:10:20"});
   }
 
   @Test
   public void testAddDays() throws IOException {
     testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT2);",
-        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-04 00:00:00"});
     testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT4);",
-        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-04 00:00:00"});
     testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT8);",
-        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-04 00:00:00"});
 
     testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT2);",
-        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-04 12:10:20"});
     testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT4);",
-        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-04 12:10:20"});
     testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT8);",
-        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2014-01-04 12:10:20"});
 
     testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT2);",
-        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-28 00:00:00"});
     testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT4);",
-        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-28 00:00:00"});
     testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT8);",
-        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-28 00:00:00"});
 
     testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT2);",
-        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-28 12:10:20"});
     testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT4);",
-        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-28 12:10:20"});
     testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT8);",
-        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
+        new String[]{"2013-11-28 12:10:20"});
   }
 
   @Test
   public void testDateTimeNow() throws IOException {
-    TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6"));
-    TimeZone systemOriginTimeZone = TimeZone.getDefault();
+    TimeZone originalTimezone = TimeZone.getDefault();
     TimeZone.setDefault(TimeZone.getTimeZone("GMT-6"));
+
+    QueryContext context = new QueryContext(getConf());
+    context.put(SessionVars.TZ, "GMT-6");
+
     try {
       Date expectedDate = new Date(System.currentTimeMillis());
 
-      testSimpleEval("select to_char(now(), 'yyyy-MM-dd');",
+      testSimpleEval(context, "select to_char(now(), 'yyyy-MM-dd');",
           new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
-      testSimpleEval("select cast(extract(year from now()) as INT4);",
+      testSimpleEval(context, "select cast(extract(year from now()) as INT4);",
           new String[]{dateFormat(expectedDate, "yyyy")});
-      testSimpleEval("select current_date();",
+      testSimpleEval(context, "select current_date();",
           new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
-      testSimpleEval("select cast(extract(hour from current_time()) as INT4);",
+      testSimpleEval(context, "select cast(extract(hour from current_time()) as INT4);",
           new String[]{String.valueOf(Integer.parseInt(dateFormat(expectedDate, "HH")))});
     } finally {
-      TajoConf.setCurrentTimeZone(originTimeZone);
-      TimeZone.setDefault(systemOriginTimeZone);
+      TimeZone.setDefault(originalTimezone);
     }
   }
 
   @Test
   public void testTimeValueKeyword() throws IOException {
-    TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6"));
-    TimeZone systemOriginTimeZone = TimeZone.getDefault();
+    TimeZone originTimeZone = TimeZone.getDefault();
     TimeZone.setDefault(TimeZone.getTimeZone("GMT-6"));
+
+    QueryContext context = new QueryContext(getConf());
+    context.put(SessionVars.TZ, "GMT-6");
+
     try {
       Date expectedDate = new Date(System.currentTimeMillis());
 
-      testSimpleEval("select to_char(current_timestamp, 'yyyy-MM-dd');",
+      testSimpleEval(context, "select to_char(current_timestamp, 'yyyy-MM-dd');",
           new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
-      testSimpleEval("select cast(extract(year from current_timestamp) as INT4);",
+      testSimpleEval(context, "select cast(extract(year from current_timestamp) as INT4);",
           new String[]{dateFormat(expectedDate, "yyyy")});
-      testSimpleEval("select current_date;",
+      testSimpleEval(context, "select current_date;",
           new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
-      testSimpleEval("select cast(extract(hour from current_time) as INT4);",
+      testSimpleEval(context, "select cast(extract(hour from current_time) as INT4);",
           new String[]{String.valueOf(Integer.parseInt(dateFormat(expectedDate, "HH")))});
     } finally {
-      TajoConf.setCurrentTimeZone(originTimeZone);
-      TimeZone.setDefault(systemOriginTimeZone);
+      TimeZone.setDefault(originTimeZone);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
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 5e6d05b..5b7641a 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
@@ -18,11 +18,9 @@
 
 package org.apache.tajo.engine.query;
 
-import org.apache.tajo.IntegrationTest;
-import org.apache.tajo.QueryTestCaseBase;
-import org.apache.tajo.TajoConstants;
+import com.google.common.collect.Lists;
+import org.apache.tajo.*;
 import org.apache.tajo.TajoProtos.QueryState;
-import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
@@ -37,6 +35,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.sql.ResultSet;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
 import static org.junit.Assert.*;
@@ -538,4 +538,46 @@ public class TestSelectQuery extends QueryTestCaseBase {
     assertResultSet(res);
     cleanupQuery(res);
   }
+
+  @Test
+  public void testTimezonedTable1() throws Exception {
+    try {
+      executeDDL("datetime_table_ddl.sql", "timezoned", new String[]{"timezoned1"});
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE IF EXISTS timezoned1");
+    }
+  }
+
+  @Test
+  public void testTimezonedTable2() throws Exception {
+    try {
+      executeDDL("datetime_table_timezoned_ddl.sql", "timezoned", new String[]{"timezoned2"});
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE IF EXISTS timezoned2");
+    }
+  }
+
+  @Test
+  public void testTimezonedTable3() throws Exception {
+    Map<String,String> sessionVars = new HashMap<String, String>();
+    sessionVars.put(SessionVars.TZ.name(), "GMT+9");
+    getClient().updateSessionVariables(sessionVars);
+
+    try {
+      executeDDL("datetime_table_timezoned_ddl.sql", "timezoned", new String[]{"timezoned3"});
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE IF EXISTS timezoned3");
+    }
+
+    getClient().unsetSessionVariables(Lists.newArrayList("TZ"));
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
----------------------------------------------------------------------
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 37454e6..48a1464 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
@@ -125,21 +125,14 @@ public class TestSortQuery extends QueryTestCaseBase {
   public final void testSortWithDate() throws Exception {
     // skip this test if catalog uses HCatalogStore.
     // It is because HCatalogStore does not support Time data type.
-    TimeZone oldTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("UTC"));
-    TimeZone systemOldTimeZone = TimeZone.getDefault();
-    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
-    try {
-      if (!testingCluster.isHCatalogStoreRunning()) {
-        // create external table table1 (col1 timestamp, col2 date, col3 time) ...
-        executeDDL("create_table_with_date_ddl.sql", "table1");
-
-        ResultSet res = executeQuery();
-        assertResultSet(res);
-        cleanupQuery(res);
-      }
-    } finally {
-      TajoConf.setCurrentTimeZone(oldTimeZone);
-      TimeZone.setDefault(systemOldTimeZone);
+
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      // create external table table1 (col1 timestamp, col2 date, col3 time) ...
+      executeDDL("create_table_with_date_ddl.sql", "table1");
+
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
index f491c63..543c17a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
+++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
@@ -32,6 +32,7 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.client.QueryClientImpl;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
@@ -105,8 +106,8 @@ public class TestResultSet {
   }
 
   @Test
-  public void test() throws IOException, SQLException {
-    TajoResultSet rs = new TajoResultSet(null, null, conf, desc);
+  public void test() throws Exception {
+    TajoResultSet rs = new TajoResultSet(TajoTestingCluster.newTajoClient(), null, conf, desc);
     ResultSetMetaData meta = rs.getMetaData();
     assertNotNull(meta);
     Schema schema = scoreSchema;
@@ -133,12 +134,6 @@ public class TestResultSet {
     // Hcatalog does not support date type, time type in hive-0.12.0
     if(util.isHCatalogStoreRunning()) return;
 
-    TimeZone tajoCurrentTimeZone = TajoConf.getCurrentTimeZone();
-    TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("UTC"));
-
-    TimeZone systemCurrentTimeZone = TimeZone.getDefault();
-    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
-
     ResultSet res = null;
     TajoClient client = TajoTestingCluster.newTajoClient();
     try {
@@ -212,8 +207,6 @@ public class TestResultSet {
       assertNotNull(timestamp);
       assertEquals("2014-01-01 10:00:00.0", timestamp.toString());
     } finally {
-      TajoConf.setCurrentTimeZone(tajoCurrentTimeZone);
-      TimeZone.setDefault(systemCurrentTimeZone);
       if (res != null) {
         res.close();
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
----------------------------------------------------------------------
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 a004baa..99baeba 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
@@ -543,9 +543,6 @@ public class TestTajoJdbc extends QueryTestCaseBase {
 
     // skip this test if catalog uses HCatalogStore.
     // It is because HCatalogStore does not support Time data type.
-    TimeZone oldTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("UTC"));
-    TimeZone systemOldTimeZone = TimeZone.getDefault();
-    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
 
     try {
       if (!testingCluster.isHCatalogStoreRunning()) {
@@ -575,9 +572,6 @@ public class TestTajoJdbc extends QueryTestCaseBase {
 
       }
     } finally {
-      TajoConf.setCurrentTimeZone(oldTimeZone);
-      TimeZone.setDefault(systemOldTimeZone);
-
       cleanupQuery(res);
       if (stmt != null) {
         stmt.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl b/tajo-core/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl
new file mode 100644
index 0000000..38e8bd9
--- /dev/null
+++ b/tajo-core/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl
@@ -0,0 +1,3 @@
+1980-4-1 01:50:30.010|1980-04-01
+80/4/1 1:50:30 AM|80/4/1
+1980 April 1 1:50:30|1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_ddl.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_ddl.sql
new file mode 100644
index 0000000..beb19b7
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_ddl.sql
@@ -0,0 +1,4 @@
+CREATE EXTERNAL TABLE ${0} (
+  t_timestamp  TIMESTAMP,
+  t_date    DATE
+) USING TEXTFILE LOCATION ${table.path}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_ddl.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_ddl.sql
new file mode 100644
index 0000000..11014d8
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_ddl.sql
@@ -0,0 +1,4 @@
+CREATE EXTERNAL TABLE ${0} (
+  t_timestamp  TIMESTAMP,
+  t_date    DATE
+) USING TEXTFILE WITH ('timezone' = 'GMT+9') LOCATION ${table.path}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable1.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable1.sql
new file mode 100644
index 0000000..38c9e90
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable1.sql
@@ -0,0 +1 @@
+SELECT * FROM timezoned1;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable2.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable2.sql
new file mode 100644
index 0000000..722fc65
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable2.sql
@@ -0,0 +1 @@
+SELECT * FROM timezoned2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable3.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable3.sql
new file mode 100644
index 0000000..32b9c3a
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testTimezonedTable3.sql
@@ -0,0 +1 @@
+SELECT * FROM timezoned3;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable1.result b/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable1.result
new file mode 100644
index 0000000..39f593b
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable1.result
@@ -0,0 +1,5 @@
+t_timestamp,t_date
+-------------------------------
+1980-04-01 01:50:30.01,1980-04-01
+1980-04-01 01:50:30,1980-04-01
+1980-04-01 01:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable2.result b/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable2.result
new file mode 100644
index 0000000..916f4be
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable2.result
@@ -0,0 +1,5 @@
+t_timestamp,t_date
+-------------------------------
+1980-03-31 16:50:30.01,1980-04-01
+1980-03-31 16:50:30,1980-04-01
+1980-03-31 16:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable3.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable3.result b/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable3.result
new file mode 100644
index 0000000..39f593b
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestSelectQuery/testTimezonedTable3.result
@@ -0,0 +1,5 @@
+t_timestamp,t_date
+-------------------------------
+1980-04-01 01:50:30.01,1980-04-01
+1980-04-01 01:50:30,1980-04-01
+1980-04-01 01:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-docs/src/main/sphinx/table_management.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/table_management.rst b/tajo-docs/src/main/sphinx/table_management.rst
index 62a8e45..2b21ddc 100644
--- a/tajo-docs/src/main/sphinx/table_management.rst
+++ b/tajo-docs/src/main/sphinx/table_management.rst
@@ -2,8 +2,11 @@
 Table Management
 ******************
 
+In Tajo, a table is a logical view of one data sources. Logically, one table consists of a logical schema, partitions, URL, and various properties. Physically, A table can be a directory in HDFS, a single file, one HBase table, or a RDBMS table. In order to make good use of Tajo, users need to understand features and physical characteristics of their physical layout. This section explains all about table management.
+
 .. toctree::
     :maxdepth: 1
 
+    table_management/table_overview
     table_management/file_formats
-    table_management/compression
+    table_management/compression
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-docs/src/main/sphinx/table_management/csv.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/table_management/csv.rst b/tajo-docs/src/main/sphinx/table_management/csv.rst
index 71313d6..37dc2ba 100644
--- a/tajo-docs/src/main/sphinx/table_management/csv.rst
+++ b/tajo-docs/src/main/sphinx/table_management/csv.rst
@@ -39,9 +39,9 @@ Now, the CSV storage format provides the following physical properties.
 * ``text.delimiter``: delimiter character. ``|`` or ``\u0001`` is usually used, and the default field delimiter is ``|``.
 * ``text.null``: NULL character. The default NULL character is an empty string ``''``. Hive's default NULL character is ``'\\N'``.
 * ``compression.codec``: Compression codec. You can enable compression feature and set specified compression algorithm. The compression algorithm used to compress files. The compression codec name should be the fully qualified class name inherited from `org.apache.hadoop.io.compress.CompressionCodec <https://hadoop.apache.org/docs/current/api/org/apache/hadoop/io/compress/CompressionCodec.html>`_. By default, compression is disabled.
-* ``csvfile.serde``: custom (De)serializer class. ``org.apache.tajo.storage.TextSerializerDeserializer`` is the default (De)serializer class.
+* ``csvfile.serde`` (deprecated): custom (De)serializer class. ``org.apache.tajo.storage.TextSerializerDeserializer`` is the default (De)serializer class.
+* ``timezone``: the time zone that the table uses for writting. When table rows are read or written, ```timestamp``` and ```time``` column values are adjusted by this timezone if it is set. Time zone can be an abbreviation form like 'PST' or 'DST'. Also, it accepts an offset-based form like 'UTC+9' or a location-based form like 'Asia/Seoul'.
 * ``text.error-tolerance.max-num``: the maximum number of permissible parsing errors. This value should be an integer value. By default, ``text.error-tolerance.max-num`` is ``0``. According to the value, parsing errors will be handled in different ways.
-
   * If ``text.error-tolerance.max-num < 0``, all parsing errors are ignored.
   * If ``text.error-tolerance.max-num == 0``, any parsing error is not allowed. If any error occurs, the query will be failed. (default)
   * If ``text.error-tolerance.max-num > 0``, the given number of parsing errors in each task will be pemissible.

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-docs/src/main/sphinx/table_management/table_overview.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/table_management/table_overview.rst b/tajo-docs/src/main/sphinx/table_management/table_overview.rst
new file mode 100644
index 0000000..bb4b827
--- /dev/null
+++ b/tajo-docs/src/main/sphinx/table_management/table_overview.rst
@@ -0,0 +1,98 @@
+*************************************
+Overview of Tajo Tables
+*************************************
+
+Overview
+========
+
+.. todo::
+
+Table Properties
+================
+All table formats provide parameters for enabling or disabling features and adjusting physical parameters.
+The ``WITH`` clause in the CREATE TABLE statement allows users to set those properties.
+
+The following example is to set a custom field delimiter, NULL character, and compression codec:
+
+.. code-block:: sql
+
+ CREATE TABLE table1 (
+  id int,
+  name text,
+  score float,
+  type text
+ ) USING CSV WITH('text.delimiter'='\u0001',
+                  'text.null'='\\N',
+                  'compression.codec'='org.apache.hadoop.io.compress.SnappyCodec');
+
+Each physical table layout has its own specialized properties. They will be addressed in :doc:`/table_management/file_formats`.
+
+
+Common Table Properties
+=======================
+
+There are some common table properties which are used in most tables.
+
+Compression
+-----------
+.. todo::
+
+Time zone
+---------
+In Tajo, a table property ``timezone`` allows users to specify a time zone that the table uses for reading or writing. 
+When each table row are read or written, ```timestamp``` and ```time``` column values are adjusted by a given time zone if it is set. Time zone can be an abbreviation form like 'PST' or 'DST'. Also, it accepts an offset-based form like 'GMT+9' or UTC+9' or a location-based form like 'Asia/Seoul'. 
+
+Each table has one time zone, and many tables can have different time zones. Internally, Tajo translates all tables data to offset-based values. So, complex queries like join with multiple time zones work well.
+
+.. note::
+
+  In many cases, offset-based forms or locaion-based forms are recommanded. In order to know the list of time zones, please refer to `List of tz database time zones <http://en.wikipedia.org/wiki/List_of_tz_database_time_zones>`_
+
+.. note::
+
+  Java 6 does not recognize many location-based time zones and an offset-based timezone using the prefix 'UTC'. We highly recommanded using the offset-based time zone using the prefix 'GMT'. In other words, you should use 'GMT-7' instead of 'UTC-7' in Java 6.
+
+How time zone works in Tajo
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+For example, consider that there is a list of delimited text lines where each rows are written with ``Asia/Seoul`` time zone (i.e., GMT + 9).
+
+.. code-block:: text
+
+  1980-4-1 01:50:30.010|1980-04-01
+  80/4/1 1:50:30 AM|80/4/1
+  1980 April 1 1:50:30|1980-04-01
+
+
+In order to register the table, we should put a table property ``'timezone'='Asia/Seoul'`` in ``CREATE TABLE`` statement as follows:
+
+.. code-block:: sql
+
+ CREATE EXTERNAL TABLE table1 (
+  t_timestamp  TIMESTAMP,
+  t_date    DATE
+ ) USING TEXTFILE WITH('text.delimiter'='|', 'timezone'='ASIA/Seoul') LOCATION '/path-to-table/'
+
+
+By default, ``tsql`` and ``TajoClient`` API use UTC time zone. So, timestamp values in the result are adjusted by the time zone offset. But, date is not adjusted because date type does not consider time zone.
+
+.. code-block:: sql
+
+  default> SELECT * FROM table1
+  t_timestamp,            t_date
+  ----------------------------------
+  1980-03-31 16:50:30.01, 1980-04-01
+  1980-03-31 16:50:30   , 1980-04-01
+  1980-03-31 16:50:30   , 1980-04-01
+
+In addition, users can set client-side time zone by setting a session variable 'TZ'. It enables a client to translate timestamp or time values to user's time zoned ones.
+
+.. code-block:: sql
+
+  default> \set TZ 'Asia/Seoul'  
+  default> SELECT * FROM table1
+  t_timestamp,            t_date
+  ----------------------------------
+  1980-04-01 01:50:30.01, 1980-04-01
+  1980-04-01 01:50:30   , 1980-04-01
+  1980-04-01 01:50:30   , 1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java
index faa058d..8f5bed6 100644
--- a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java
+++ b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java
@@ -31,12 +31,14 @@ public class TajoMetaDataResultSet extends TajoResultSetBase {
   private List<MetaDataTuple> values;
 
   public TajoMetaDataResultSet(Schema schema, List<MetaDataTuple> values) {
+    super(null);
     init();
     this.schema = schema;
     setDataTuples(values);
   }
 
   public TajoMetaDataResultSet(List<String> columns, List<Type> types, List<MetaDataTuple> values) {
+    super(null);
     init();
     schema = new Schema();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
index 406cdfc..62ba33c 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
@@ -19,6 +19,8 @@
 package org.apache.tajo.plan;
 
 import com.google.common.collect.Sets;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
@@ -43,6 +45,7 @@ import org.apache.tajo.util.datetime.TimeMeta;
 
 import java.util.Set;
 import java.util.Stack;
+import java.util.TimeZone;
 
 import static org.apache.tajo.algebra.WindowSpec.WindowFrameEndBoundType;
 import static org.apache.tajo.algebra.WindowSpec.WindowFrameStartBoundType;
@@ -64,11 +67,16 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
   }
 
   static class Context {
+    OverridableConf queryContext;
+    TimeZone timeZone;
     LogicalPlan plan;
     LogicalPlan.QueryBlock currentBlock;
     NameResolvingMode columnRsvLevel;
 
     public Context(LogicalPlanner.PlanContext planContext, NameResolvingMode colRsvLevel) {
+      this.queryContext = planContext.queryContext;
+      this.timeZone = planContext.timeZone;
+
       this.plan = planContext.plan;
       this.currentBlock = planContext.queryBlock;
       this.columnRsvLevel = colRsvLevel;
@@ -95,7 +103,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
    * @param rhs right hand side term
    * @return a pair including left/right hand side terms
    */
-  public static Pair<EvalNode, EvalNode> convertTypesIfNecessary(EvalNode lhs, EvalNode rhs) {
+  private static Pair<EvalNode, EvalNode> convertTypesIfNecessary(Context ctx, EvalNode lhs, EvalNode rhs) {
     Type lhsType = lhs.getValueType().getType();
     Type rhsType = rhs.getValueType().getType();
 
@@ -108,10 +116,10 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     if (toBeCasted != null) { // if not null, one of either should be converted to another type.
       // Overwrite lhs, rhs, or both with cast expression.
       if (lhsType != toBeCasted) {
-        lhs = convertType(lhs, CatalogUtil.newSimpleDataType(toBeCasted));
+        lhs = convertType(ctx, lhs, CatalogUtil.newSimpleDataType(toBeCasted));
       }
       if (rhsType != toBeCasted) {
-        rhs = convertType(rhs, CatalogUtil.newSimpleDataType(toBeCasted));
+        rhs = convertType(ctx, rhs, CatalogUtil.newSimpleDataType(toBeCasted));
       }
     }
 
@@ -126,7 +134,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
    * @param toType target type
    * @return type converted expression.
    */
-  private static EvalNode convertType(EvalNode evalNode, DataType toType) {
+  private static EvalNode convertType(Context ctx, EvalNode evalNode, DataType toType) {
 
     // if original and toType is the same, we don't need type conversion.
     if (evalNode.getValueType().equals(toType)) {
@@ -140,9 +148,9 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     if (evalNode.getType() == EvalType.BETWEEN) {
       BetweenPredicateEval between = (BetweenPredicateEval) evalNode;
 
-      between.setPredicand(convertType(between.getPredicand(), toType));
-      between.setBegin(convertType(between.getBegin(), toType));
-      between.setEnd(convertType(between.getEnd(), toType));
+      between.setPredicand(convertType(ctx, between.getPredicand(), toType));
+      between.setBegin(convertType(ctx, between.getBegin(), toType));
+      between.setEnd(convertType(ctx, between.getEnd(), toType));
 
       return between;
 
@@ -150,11 +158,11 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
 
       CaseWhenEval caseWhenEval = (CaseWhenEval) evalNode;
       for (CaseWhenEval.IfThenEval ifThen : caseWhenEval.getIfThenEvals()) {
-        ifThen.setResult(convertType(ifThen.getResult(), toType));
+        ifThen.setResult(convertType(ctx, ifThen.getResult(), toType));
       }
 
       if (caseWhenEval.hasElse()) {
-        caseWhenEval.setElseResult(convertType(caseWhenEval.getElse(), toType));
+        caseWhenEval.setElseResult(convertType(ctx, caseWhenEval.getElse(), toType));
       }
 
       return caseWhenEval;
@@ -166,7 +174,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
       Datum[] convertedDatum = new Datum[datums.length];
 
       for (int i = 0; i < datums.length; i++) {
-        convertedDatum[i] = DatumFactory.cast(datums[i], toType);
+        convertedDatum[i] = DatumFactory.cast(datums[i], toType, ctx.timeZone);
       }
 
       RowConstantEval convertedRowConstant = new RowConstantEval(convertedDatum);
@@ -175,11 +183,11 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
 
     } else if (evalNode.getType() == EvalType.CONST) {
       ConstEval original = (ConstEval) evalNode;
-      ConstEval newConst = new ConstEval(DatumFactory.cast(original.getValue(), toType));
+      ConstEval newConst = new ConstEval(DatumFactory.cast(original.getValue(), toType, ctx.timeZone));
       return newConst;
 
     } else {
-      return new CastEval(evalNode, toType);
+      return new CastEval(ctx.queryContext, evalNode, toType);
     }
   }
 
@@ -279,7 +287,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
       throw new IllegalStateException("Wrong Expr Type: " + expr.getType());
     }
 
-    return createBinaryNode(evalType, left, right);
+    return createBinaryNode(ctx, evalType, left, right);
   }
 
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////
@@ -308,7 +316,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
         between.isSymmetric(),
         predicand, begin, end);
 
-    betweenEval = (BetweenPredicateEval) convertType(betweenEval, widestType);
+    betweenEval = (BetweenPredicateEval) convertType(ctx, betweenEval, widestType);
     return betweenEval;
   }
 
@@ -342,7 +350,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     assertEval(widestType != null, "Invalid Type Conversion for CaseWhen");
 
     // implicit type conversion
-    caseWhenEval = (CaseWhenEval) convertType(caseWhenEval, widestType);
+    caseWhenEval = (CaseWhenEval) convertType(ctx, caseWhenEval, widestType);
 
     return caseWhenEval;
   }
@@ -362,7 +370,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     RowConstantEval rowConstantEval = (RowConstantEval) visit(ctx, stack, expr.getInValue());
     stack.pop();
 
-    Pair<EvalNode, EvalNode> pair = convertTypesIfNecessary(lhs, rowConstantEval);
+    Pair<EvalNode, EvalNode> pair = convertTypesIfNecessary(ctx, lhs, rowConstantEval);
 
     return new InEval(pair.getFirst(), (RowConstantEval) pair.getSecond(), expr.isNot());
   }
@@ -415,10 +423,10 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     stack.pop();
 
     if (lhs.getValueType().getType() != Type.TEXT) {
-      lhs = convertType(lhs, CatalogUtil.newSimpleDataType(Type.TEXT));
+      lhs = convertType(ctx, lhs, CatalogUtil.newSimpleDataType(Type.TEXT));
     }
     if (rhs.getValueType().getType() != Type.TEXT) {
-      rhs = convertType(rhs, CatalogUtil.newSimpleDataType(Type.TEXT));
+      rhs = convertType(ctx, rhs, CatalogUtil.newSimpleDataType(Type.TEXT));
     }
 
     return new BinaryEval(EvalType.CONCATENATE, lhs, rhs);
@@ -448,8 +456,8 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
   // Arithmetic Operators
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////
 
-  private static BinaryEval createBinaryNode(EvalType type, EvalNode lhs, EvalNode rhs) {
-    Pair<EvalNode, EvalNode> pair = convertTypesIfNecessary(lhs, rhs); // implicit type conversion if necessary
+  private static BinaryEval createBinaryNode(Context ctx, EvalType type, EvalNode lhs, EvalNode rhs) {
+    Pair<EvalNode, EvalNode> pair = convertTypesIfNecessary(ctx, lhs, rhs); // implicit type conversion if necessary
     return new BinaryEval(type, pair.getFirst(), pair.getSecond());
   }
 
@@ -460,7 +468,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     EvalNode right = visit(ctx, stack, expr.getRight());
     stack.pop();
 
-    return createBinaryNode(EvalType.PLUS, left, right);
+    return createBinaryNode(ctx, EvalType.PLUS, left, right);
   }
 
   @Override
@@ -470,7 +478,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     EvalNode right = visit(ctx, stack, expr.getRight());
     stack.pop();
 
-    return createBinaryNode(EvalType.MINUS, left, right);
+    return createBinaryNode(ctx, EvalType.MINUS, left, right);
   }
 
   @Override
@@ -480,7 +488,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     EvalNode right = visit(ctx, stack, expr.getRight());
     stack.pop();
 
-    return createBinaryNode(EvalType.MULTIPLY, left, right);
+    return createBinaryNode(ctx, EvalType.MULTIPLY, left, right);
   }
 
   @Override
@@ -490,7 +498,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     EvalNode right = visit(ctx, stack, expr.getRight());
     stack.pop();
 
-    return createBinaryNode(EvalType.DIVIDE, left, right);
+    return createBinaryNode(ctx, EvalType.DIVIDE, left, right);
   }
 
   @Override
@@ -500,7 +508,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     EvalNode right = visit(ctx, stack, expr.getRight());
     stack.pop();
 
-    return createBinaryNode(EvalType.MODULAR, left, right);
+    return createBinaryNode(ctx, EvalType.MODULAR, left, right);
   }
 
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////
@@ -584,14 +592,14 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
         } else {
           lastDataType = CatalogUtil.newSimpleDataType(CatalogUtil.getPrimitiveTypeOf(lastDataType.getType()));
         }
-        givenArgs[i] = convertType(givenArgs[i], lastDataType);
+        givenArgs[i] = convertType(ctx, givenArgs[i], lastDataType);
       }
     } else {
       assertEval(funcDesc.getParamTypes().length == givenArgs.length,
           "The number of parameters is mismatched to the function definition: " + funcDesc.toString());
       // According to our function matching method, each given argument can be casted to the definition parameter.
       for (int i = 0; i < givenArgs.length; i++) {
-        givenArgs[i] = convertType(givenArgs[i], funcDesc.getParamTypes()[i]);
+        givenArgs[i] = convertType(ctx, givenArgs[i], funcDesc.getParamTypes()[i]);
       }
     }
 
@@ -600,7 +608,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
       FunctionType functionType = funcDesc.getFuncType();
       if (functionType == FunctionType.GENERAL
           || functionType == FunctionType.UDF) {
-        return new GeneralFunctionEval(funcDesc, (GeneralFunction) funcDesc.newInstance(), givenArgs);
+        return new GeneralFunctionEval(ctx.queryContext, funcDesc, (GeneralFunction) funcDesc.newInstance(), givenArgs);
       } else if (functionType == FunctionType.AGGREGATION
           || functionType == FunctionType.UDA) {
         if (!ctx.currentBlock.hasNode(NodeType.GROUP_BY)) {
@@ -768,11 +776,23 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
   public EvalNode visitCastExpr(Context ctx, Stack<Expr> stack, CastExpr expr) throws PlanningException {
     EvalNode child = super.visitCastExpr(ctx, stack, expr);
 
-    if (child.getType() == EvalType.CONST) { // if it is a casting operation for a constant value
-      ConstEval constEval = (ConstEval) child; // it will be pre-computed and casted to a constant value
-      return new ConstEval(DatumFactory.cast(constEval.getValue(), LogicalPlanner.convertDataType(expr.getTarget())));
+    // if it is a casting operation for a constant value, it will be pre-computed and casted to a constant value.
+
+    if (child.getType() == EvalType.CONST) {
+      ConstEval constEval = (ConstEval) child;
+
+      // some cast operation may require earlier evaluation with timezone.
+      TimeZone tz = null;
+      if (ctx.queryContext.containsKey(SessionVars.TZ)) {
+        String tzId = ctx.queryContext.get(SessionVars.TZ);
+        tz = TimeZone.getTimeZone(tzId);
+      }
+
+      return new ConstEval(
+          DatumFactory.cast(constEval.getValue(), LogicalPlanner.convertDataType(expr.getTarget()), tz));
+
     } else {
-      return new CastEval(child, LogicalPlanner.convertDataType(expr.getTarget()));
+      return new CastEval(ctx.queryContext, child, LogicalPlanner.convertDataType(expr.getTarget()));
     }
   }
 
@@ -838,7 +858,11 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
 
     TimeMeta tm = new TimeMeta();
     DateTimeUtil.toJulianTimeMeta(timestamp, tm);
-    DateTimeUtil.toUTCTimezone(tm);
+
+    if (ctx.queryContext.containsKey(SessionVars.TZ)) {
+      TimeZone tz = TimeZone.getTimeZone(ctx.queryContext.get(SessionVars.TZ));
+      DateTimeUtil.toUTCTimezone(tm, tz);
+    }
 
     return new ConstEval(new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm)));
   }
@@ -864,7 +888,11 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     }
     TimeDatum timeDatum = new TimeDatum(time);
     TimeMeta tm = timeDatum.toTimeMeta();
-    DateTimeUtil.toUTCTimezone(tm);
+
+    if (ctx.queryContext.containsKey(SessionVars.TZ)) {
+      TimeZone tz = TimeZone.getTimeZone(ctx.queryContext.get(SessionVars.TZ));
+      DateTimeUtil.toUTCTimezone(tm, tz);
+    }
 
     return new ConstEval(new TimeDatum(DateTimeUtil.toTime(tm)));
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
index 69c0e4b..f21bbb5 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
@@ -83,6 +83,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     LogicalPlan plan;
     QueryBlock queryBlock;
     EvalTreeOptimizer evalOptimizer;
+    TimeZone timeZone;
     boolean debugOrUnitTests;
 
     public PlanContext(OverridableConf context, LogicalPlan plan, QueryBlock block, EvalTreeOptimizer evalOptimizer,
@@ -91,6 +92,13 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       this.plan = plan;
       this.queryBlock = block;
       this.evalOptimizer = evalOptimizer;
+
+      // session's time zone
+      if (context.containsKey(SessionVars.TZ)) {
+        String timezoneId = context.get(SessionVars.TZ);
+        timeZone = TimeZone.getTimeZone(timezoneId);
+      }
+
       this.debugOrUnitTests = debugOrUnitTests;
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java
index d625a11..700913e 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java
@@ -19,19 +19,31 @@
 package org.apache.tajo.plan.expr;
 
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.TUtil;
+
+import java.util.TimeZone;
 
 import static org.apache.tajo.common.TajoDataTypes.DataType;
 
 public class CastEval extends UnaryEval {
   @Expose private DataType target;
+  @Expose private TimeZone timezone;
 
-  public CastEval(EvalNode operand, DataType target) {
+  public CastEval(OverridableConf context, EvalNode operand, DataType target) {
     super(EvalType.CAST, operand);
     this.target = target;
+
+    if (context.containsKey(SessionVars.TZ)) {
+      String timezoneId = context.get(SessionVars.TZ);
+      timezone = TimeZone.getTimeZone(timezoneId);
+    }
   }
 
   public EvalNode getOperand() {
@@ -43,6 +55,14 @@ public class CastEval extends UnaryEval {
     return target;
   }
 
+  public boolean hasTimeZone() {
+    return this.timezone != null;
+  }
+
+  public TimeZone getTimezone() {
+    return this.timezone;
+  }
+
   @Override
   public String getName() {
     return target.getType().name();
@@ -54,7 +74,7 @@ public class CastEval extends UnaryEval {
       return operandDatum;
     }
 
-    return DatumFactory.cast(operandDatum, target);
+    return DatumFactory.cast(operandDatum, target, timezone);
   }
 
   public String toString() {
@@ -66,7 +86,9 @@ public class CastEval extends UnaryEval {
     boolean valid = obj != null && obj instanceof CastEval;
     if (valid) {
       CastEval another = (CastEval) obj;
-      return child.equals(another.child) && target.equals(another.target);
+      return child.equals(another.child) &&
+          target.equals(another.target) &&
+          TUtil.checkEquals(timezone, another.timezone);
     } else {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java
index e28e5f3..0c6f1e2 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java
@@ -20,6 +20,7 @@ package org.apache.tajo.plan.expr;
 
 import com.google.common.base.Objects;
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.datum.Datum;
@@ -28,14 +29,17 @@ import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
 import org.apache.tajo.util.TUtil;
 
+import javax.annotation.Nullable;
+
 public class GeneralFunctionEval extends FunctionEval {
   @Expose protected GeneralFunction instance;
   private Tuple params = null;
 
-	public GeneralFunctionEval(FunctionDesc desc, GeneralFunction instance, EvalNode[] givenArgs) {
+	public GeneralFunctionEval(@Nullable OverridableConf queryContext, FunctionDesc desc, GeneralFunction instance,
+                             EvalNode[] givenArgs) {
 		super(EvalType.FUNCTION, desc, givenArgs);
 		this.instance = instance;
-    this.instance.init(getParamType());
+    this.instance.init(queryContext, getParamType());
   }
 
   /* (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
index 006449f..39db5c6 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.plan.function;
 
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -36,7 +37,8 @@ public abstract class GeneralFunction extends Function implements GsonObject {
   /**
    * This method gives hints to an actual function instance.
    */
-  public void init(FunctionEval.ParamType [] paramTypes) {}
+  @SuppressWarnings("unused")
+  public void init(OverridableConf queryContext, FunctionEval.ParamType [] paramTypes) {}
 
   public abstract Datum eval(Tuple params);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoDeserializer.java
index e6d54b1..89b4fc0 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoDeserializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoDeserializer.java
@@ -20,6 +20,7 @@ package org.apache.tajo.plan.serder;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.exception.NoSuchFunctionException;
@@ -43,7 +44,7 @@ import java.util.*;
  */
 public class EvalTreeProtoDeserializer {
 
-  public static EvalNode deserialize(PlanProto.EvalTree tree) {
+  public static EvalNode deserialize(OverridableConf context, PlanProto.EvalTree tree) {
     Map<Integer, EvalNode> evalNodeMap = Maps.newHashMap();
 
     // sort serialized eval nodes in an ascending order of their IDs.
@@ -79,7 +80,7 @@ public class EvalTreeProtoDeserializer {
           current = new IsNullEval(unaryProto.getNegative(), child);
           break;
         case CAST:
-          current = new CastEval(child, unaryProto.getCastingType());
+          current = new CastEval(context, child, unaryProto.getCastingType());
           break;
         case SIGNED:
           current = new SignedEval(unaryProto.getNegative(), child);
@@ -153,7 +154,7 @@ public class EvalTreeProtoDeserializer {
           funcDesc = new FunctionDesc(funcProto.getFuncion());
           if (type == EvalType.FUNCTION) {
             GeneralFunction instance = (GeneralFunction) funcDesc.newInstance();
-            current = new GeneralFunctionEval(new FunctionDesc(funcProto.getFuncion()), instance, params);
+            current = new GeneralFunctionEval(context, new FunctionDesc(funcProto.getFuncion()), instance, params);
           } else if (type == EvalType.AGG_FUNCTION || type == EvalType.WINDOW_FUNCTION) {
             AggFunction instance = (AggFunction) funcDesc.newInstance();
             if (type == EvalType.AGG_FUNCTION) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoSerializer.java
index 9f22c20..92a245f 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoSerializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalTreeProtoSerializer.java
@@ -111,6 +111,9 @@ public class EvalTreeProtoSerializer
     } else if (unary.getType() == EvalType.CAST) {
       CastEval castEval = (CastEval) unary;
       unaryBuilder.setCastingType(castEval.getValueType());
+      if (castEval.hasTimeZone()) {
+        unaryBuilder.setTimezone(castEval.getTimezone().getID());
+      }
     }
 
     // registering itself and building EvalNode

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
index 8cafbd0..befeb2a 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
@@ -36,6 +36,7 @@ import org.apache.tajo.util.TUtil;
 
 import java.lang.reflect.Type;
 import java.util.Map;
+import java.util.TimeZone;
 
 public class PlanGsonHelper {
   private static Gson gson;
@@ -56,6 +57,7 @@ public class PlanGsonHelper {
     adapters.put(AggFunction.class, new FunctionAdapter());
     adapters.put(Datum.class, new DatumAdapter());
     adapters.put(DataType.class, new DataTypeAdapter());
+    adapters.put(TimeZone.class, new TimeZoneGsonSerdeAdapter());
 
     return adapters;
 	}

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
index c55c203..8a9e1ca 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
@@ -791,6 +791,10 @@ public class PlannerUtil {
       if (sessionVars.containsKey(SessionVars.NULL_CHAR)) {
         tableProperties.set(StorageConstants.TEXT_NULL, sessionVars.get(SessionVars.NULL_CHAR));
       }
+
+      if (sessionVars.containsKey(SessionVars.TZ)) {
+        tableProperties.set(StorageConstants.TIMEZONE, sessionVars.get(SessionVars.TZ));
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-plan/src/main/proto/Plan.proto
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto
index 8639117..0f82d87 100644
--- a/tajo-plan/src/main/proto/Plan.proto
+++ b/tajo-plan/src/main/proto/Plan.proto
@@ -150,6 +150,7 @@ message UnaryEval {
   required int32 child_id = 1;
   optional DataType castingType = 2;
   optional bool negative = 3;
+  optional string timezone = 4;
 }
 
 message BinaryEval {

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
index 094d285..ce7b11d 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
@@ -31,14 +31,13 @@ import org.apache.tajo.util.NumberUtil;
 import java.io.IOException;
 import java.io.OutputStream;
 
-//Compatibility with Apache Hive
+// Compatibility with Apache Hive
 @Deprecated
 public class TextSerializerDeserializer implements SerializerDeserializer {
   public static final byte[] trueBytes = "true".getBytes();
   public static final byte[] falseBytes = "false".getBytes();
   private ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
 
-
   @Override
   public int serialize(Column col, Datum datum, OutputStream out, byte[] nullCharacters) throws IOException {
 
@@ -86,12 +85,12 @@ public class TextSerializerDeserializer implements SerializerDeserializer {
         out.write(bytes);
         break;
       case TIME:
-        bytes = ((TimeDatum)datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        bytes = ((TimeDatum)datum).asChars(TajoConf.getSystemTimezone(), true).getBytes();
         length = bytes.length;
         out.write(bytes);
         break;
       case TIMESTAMP:
-        bytes = ((TimestampDatum)datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        bytes = ((TimestampDatum)datum).asChars(TajoConf.getSystemTimezone(), true).getBytes();
         length = bytes.length;
         out.write(bytes);
         break;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
index f2eebc6..1599f62 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
@@ -45,7 +45,7 @@ public class CSVLineDeserializer extends TextLineDeserializer {
     }
     nullChars = TextLineSerDe.getNullChars(meta);
 
-    fieldSerDer = new TextFieldSerializerDeserializer();
+    fieldSerDer = new TextFieldSerializerDeserializer(meta);
   }
 
   public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException, TextLineParsingError {

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
index 7397000..c0fc18f 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
@@ -45,7 +45,7 @@ public class CSVLineSerializer extends TextLineSerializer {
     delimiter = CSVLineSerDe.getFieldDelimiter(meta);
     columnNum = schema.size();
 
-    serde = new TextFieldSerializerDeserializer();
+    serde = new TextFieldSerializerDeserializer(meta);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
index 95d0407..ae7565d 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
@@ -22,25 +22,36 @@ import com.google.protobuf.Message;
 import io.netty.buffer.ByteBuf;
 import io.netty.util.CharsetUtil;
 import org.apache.commons.codec.binary.Base64;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.*;
 import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
 import org.apache.tajo.storage.FieldSerializerDeserializer;
+import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.NumberUtil;
 
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.charset.CharsetDecoder;
+import java.util.TimeZone;
 
-//Compatibility with Apache Hive
 public class TextFieldSerializerDeserializer implements FieldSerializerDeserializer {
   public static final byte[] trueBytes = "true".getBytes();
   public static final byte[] falseBytes = "false".getBytes();
   private static ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
   private final CharsetDecoder decoder = CharsetUtil.getDecoder(CharsetUtil.UTF_8);
 
+  private final boolean hasTimezone;
+  private final TimeZone timezone;
+
+  public TextFieldSerializerDeserializer(TableMeta meta) {
+    hasTimezone = meta.containsOption(StorageConstants.TIMEZONE);
+    timezone = TimeZone.getTimeZone(meta.getOption(StorageConstants.TIMEZONE, TajoConstants.DEFAULT_SYSTEM_TIMEZONE));
+  }
+
   private static boolean isNull(ByteBuf val, ByteBuf nullBytes) {
     return !val.isReadable() || nullBytes.equals(val);
   }
@@ -50,7 +61,8 @@ public class TextFieldSerializerDeserializer implements FieldSerializerDeseriali
   }
 
   @Override
-  public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) throws IOException {
+  public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars)
+      throws IOException {
     byte[] bytes;
     int length = 0;
     TajoDataTypes.DataType dataType = col.getDataType();
@@ -95,12 +107,20 @@ public class TextFieldSerializerDeserializer implements FieldSerializerDeseriali
         out.write(bytes);
         break;
       case TIME:
-        bytes = ((TimeDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        if (hasTimezone) {
+          bytes = ((TimeDatum) datum).asChars(timezone, true).getBytes();
+        } else {
+          bytes = datum.asTextBytes();
+        }
         length = bytes.length;
         out.write(bytes);
         break;
       case TIMESTAMP:
-        bytes = ((TimestampDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        if (hasTimezone) {
+          bytes = ((TimestampDatum) datum).asChars(timezone, true).getBytes();
+        } else {
+          bytes = datum.asTextBytes();
+        }
         length = bytes.length;
         out.write(bytes);
         break;
@@ -178,12 +198,22 @@ public class TextFieldSerializerDeserializer implements FieldSerializerDeseriali
               decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
           break;
         case TIME:
-          datum = DatumFactory.createTime(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          if (hasTimezone) {
+            datum = DatumFactory.createTime(
+                decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString(), timezone);
+          } else {
+            datum = DatumFactory.createTime(
+                decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          }
           break;
         case TIMESTAMP:
-          datum = DatumFactory.createTimestamp(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          if (hasTimezone) {
+            datum = DatumFactory.createTimestamp(
+                decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString(), timezone);
+          } else {
+            datum = DatumFactory.createTimestamp(
+                decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          }
           break;
         case INTERVAL:
           datum = DatumFactory.createInterval(


[07/10] tajo git commit: TAJO-1229: rename tajo-yarn-pullserver to tajo-pullserver.

Posted by ji...@apache.org.
TAJO-1229: rename tajo-yarn-pullserver to tajo-pullserver.

Closes #284


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

Branch: refs/heads/index_support
Commit: b5aa780460fcfbf657541ee6c94d41b34b1b24b9
Parents: facd1dd
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 8 17:27:16 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 8 17:27:16 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 pom.xml                                         |   2 +-
 tajo-pullserver/pom.xml                         | 146 ++++
 .../tajo/pullserver/FadvisedChunkedFile.java    |  81 ++
 .../tajo/pullserver/FadvisedFileRegion.java     | 170 ++++
 .../FileAccessForbiddenException.java           |  40 +
 .../tajo/pullserver/FileCloseListener.java      |  53 ++
 .../tajo/pullserver/HttpDataServerHandler.java  | 245 ++++++
 .../HttpDataServerPipelineFactory.java          |  56 ++
 .../org/apache/tajo/pullserver/HttpUtil.java    |  69 ++
 .../tajo/pullserver/PullServerAuxService.java   | 654 +++++++++++++++
 .../apache/tajo/pullserver/PullServerUtil.java  |  90 +++
 .../apache/tajo/pullserver/TajoPullServer.java  |  73 ++
 .../tajo/pullserver/TajoPullServerService.java  | 808 +++++++++++++++++++
 .../retriever/AdvancedDataRetriever.java        | 126 +++
 .../pullserver/retriever/DataRetriever.java     |  29 +
 .../retriever/DirectoryRetriever.java           |  56 ++
 .../tajo/pullserver/retriever/FileChunk.java    |  81 ++
 .../pullserver/retriever/RetrieverHandler.java  |  33 +
 tajo-yarn-pullserver/pom.xml                    | 146 ----
 .../tajo/pullserver/FadvisedChunkedFile.java    |  81 --
 .../tajo/pullserver/FadvisedFileRegion.java     | 170 ----
 .../FileAccessForbiddenException.java           |  40 -
 .../tajo/pullserver/FileCloseListener.java      |  53 --
 .../tajo/pullserver/HttpDataServerHandler.java  | 245 ------
 .../HttpDataServerPipelineFactory.java          |  56 --
 .../org/apache/tajo/pullserver/HttpUtil.java    |  69 --
 .../tajo/pullserver/PullServerAuxService.java   | 654 ---------------
 .../apache/tajo/pullserver/PullServerUtil.java  |  90 ---
 .../apache/tajo/pullserver/TajoPullServer.java  |  73 --
 .../tajo/pullserver/TajoPullServerService.java  | 808 -------------------
 .../retriever/AdvancedDataRetriever.java        | 126 ---
 .../pullserver/retriever/DataRetriever.java     |  29 -
 .../retriever/DirectoryRetriever.java           |  56 --
 .../tajo/pullserver/retriever/FileChunk.java    |  81 --
 .../pullserver/retriever/RetrieverHandler.java  |  33 -
 36 files changed, 2814 insertions(+), 2811 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index acc72b9..c84992b 100644
--- a/CHANGES
+++ b/CHANGES
@@ -154,6 +154,9 @@ Release 0.9.1 - unreleased
 
   TASKS
 
+    TAJO-1229: rename tajo-yarn-pullserver to tajo-pullserver.
+    (hyunsik)
+
     TAJO-1157: Required Java version in tutorial doc needs to be updated.
     (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3dca9c0..62e03f7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -89,7 +89,7 @@
     <module>tajo-client</module>
     <module>tajo-jdbc</module>
     <module>tajo-storage</module>
-    <module>tajo-yarn-pullserver</module>
+    <module>tajo-pullserver</module>
     <module>tajo-dist</module>
     <module>tajo-thirdparty/asm</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-pullserver/pom.xml b/tajo-pullserver/pom.xml
new file mode 100644
index 0000000..a7644a1
--- /dev/null
+++ b/tajo-pullserver/pom.xml
@@ -0,0 +1,146 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.9.1-SNAPSHOT</version>
+    <relativePath>../tajo-project</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <name>Tajo Core PullServer</name>
+  <artifactId>tajo-yarn-pullserver</artifactId>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-rpc</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-shuffle</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+        <version>2.4</version>
+        <configuration>
+          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+    </plugins>
+  </reporting>
+
+</project>

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java
new file mode 100644
index 0000000..b0b8d18
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedChunkedFile.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.jboss.netty.handler.stream.ChunkedFile;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+public class FadvisedChunkedFile extends ChunkedFile {
+
+  private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
+
+  private final boolean manageOsCache;
+  private final int readaheadLength;
+  private final ReadaheadPool readaheadPool;
+  private final FileDescriptor fd;
+  private final String identifier;
+
+  private ReadaheadPool.ReadaheadRequest readaheadRequest;
+
+  public FadvisedChunkedFile(RandomAccessFile file, long position, long count,
+                             int chunkSize, boolean manageOsCache, int readaheadLength,
+                             ReadaheadPool readaheadPool, String identifier) throws IOException {
+    super(file, position, count, chunkSize);
+    this.manageOsCache = manageOsCache;
+    this.readaheadLength = readaheadLength;
+    this.readaheadPool = readaheadPool;
+    this.fd = file.getFD();
+    this.identifier = identifier;
+  }
+
+  @Override
+  public Object nextChunk() throws Exception {
+    if (PullServerUtil.isNativeIOPossible() && manageOsCache && readaheadPool != null) {
+      readaheadRequest = readaheadPool
+          .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength,
+              getEndOffset(), readaheadRequest);
+    }
+    return super.nextChunk();
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (readaheadRequest != null) {
+      readaheadRequest.cancel();
+    }
+    if (PullServerUtil.isNativeIOPossible() && manageOsCache && getEndOffset() - getStartOffset() > 0) {
+      try {
+        PullServerUtil.posixFadviseIfPossible(identifier,
+            fd,
+            getStartOffset(), getEndOffset() - getStartOffset(),
+            NativeIO.POSIX.POSIX_FADV_DONTNEED);
+      } catch (Throwable t) {
+        LOG.warn("Failed to manage OS cache for " + identifier, t);
+      }
+    }
+    super.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java
new file mode 100644
index 0000000..18cf4b6
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FadvisedFileRegion.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.jboss.netty.channel.DefaultFileRegion;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+
+public class FadvisedFileRegion extends DefaultFileRegion {
+
+  private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class);
+
+  private final boolean manageOsCache;
+  private final int readaheadLength;
+  private final ReadaheadPool readaheadPool;
+  private final FileDescriptor fd;
+  private final String identifier;
+  private final long count;
+  private final long position;
+  private final int shuffleBufferSize;
+  private final boolean shuffleTransferToAllowed;
+  private final FileChannel fileChannel;
+
+  private ReadaheadPool.ReadaheadRequest readaheadRequest;
+  public static final int DEFAULT_SHUFFLE_BUFFER_SIZE = 128 * 1024;
+
+  public FadvisedFileRegion(RandomAccessFile file, long position, long count,
+                            boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool,
+                            String identifier) throws IOException {
+    this(file, position, count, manageOsCache, readaheadLength, readaheadPool,
+        identifier, DEFAULT_SHUFFLE_BUFFER_SIZE, true);
+  }
+
+  public FadvisedFileRegion(RandomAccessFile file, long position, long count,
+                            boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool,
+                            String identifier, int shuffleBufferSize,
+                            boolean shuffleTransferToAllowed) throws IOException {
+    super(file.getChannel(), position, count);
+    this.manageOsCache = manageOsCache;
+    this.readaheadLength = readaheadLength;
+    this.readaheadPool = readaheadPool;
+    this.fd = file.getFD();
+    this.identifier = identifier;
+    this.fileChannel = file.getChannel();
+    this.count = count;
+    this.position = position;
+    this.shuffleBufferSize = shuffleBufferSize;
+    this.shuffleTransferToAllowed = shuffleTransferToAllowed;
+  }
+
+  @Override
+  public long transferTo(WritableByteChannel target, long position)
+      throws IOException {
+    if (PullServerUtil.isNativeIOPossible() && manageOsCache && readaheadPool != null) {
+      readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
+          getPosition() + position, readaheadLength,
+          getPosition() + getCount(), readaheadRequest);
+    }
+
+    if(this.shuffleTransferToAllowed) {
+      return super.transferTo(target, position);
+    } else {
+      return customShuffleTransfer(target, position);
+    }
+  }
+
+  /**
+   * This method transfers data using local buffer. It transfers data from
+   * a disk to a local buffer in memory, and then it transfers data from the
+   * buffer to the target. This is used only if transferTo is disallowed in
+   * the configuration file. super.TransferTo does not perform well on Windows
+   * due to a small IO request generated. customShuffleTransfer can control
+   * the size of the IO requests by changing the size of the intermediate
+   * buffer.
+   */
+  @VisibleForTesting
+  long customShuffleTransfer(WritableByteChannel target, long position)
+      throws IOException {
+    long actualCount = this.count - position;
+    if (actualCount < 0 || position < 0) {
+      throw new IllegalArgumentException(
+          "position out of range: " + position +
+              " (expected: 0 - " + (this.count - 1) + ')');
+    }
+    if (actualCount == 0) {
+      return 0L;
+    }
+
+    long trans = actualCount;
+    int readSize;
+    ByteBuffer byteBuffer = ByteBuffer.allocate(this.shuffleBufferSize);
+
+    while(trans > 0L &&
+        (readSize = fileChannel.read(byteBuffer, this.position+position)) > 0) {
+      //adjust counters and buffer limit
+      if(readSize < trans) {
+        trans -= readSize;
+        position += readSize;
+        byteBuffer.flip();
+      } else {
+        //We can read more than we need if the actualCount is not multiple
+        //of the byteBuffer size and file is big enough. In that case we cannot
+        //use flip method but we need to set buffer limit manually to trans.
+        byteBuffer.limit((int)trans);
+        byteBuffer.position(0);
+        position += trans;
+        trans = 0;
+      }
+
+      //write data to the target
+      while(byteBuffer.hasRemaining()) {
+        target.write(byteBuffer);
+      }
+
+      byteBuffer.clear();
+    }
+
+    return actualCount - trans;
+  }
+
+
+  @Override
+  public void releaseExternalResources() {
+    if (readaheadRequest != null) {
+      readaheadRequest.cancel();
+    }
+    super.releaseExternalResources();
+  }
+
+  /**
+   * Call when the transfer completes successfully so we can advise the OS that
+   * we don't need the region to be cached anymore.
+   */
+  public void transferSuccessful() {
+    if (PullServerUtil.isNativeIOPossible() && manageOsCache && getCount() > 0) {
+      try {
+        PullServerUtil.posixFadviseIfPossible(identifier, fd, getPosition(), getCount(),
+            NativeIO.POSIX.POSIX_FADV_DONTNEED);
+      } catch (Throwable t) {
+        LOG.warn("Failed to manage OS cache for " + identifier, t);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java
new file mode 100644
index 0000000..c703f6f
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileAccessForbiddenException.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import java.io.IOException;
+
+public class FileAccessForbiddenException extends IOException {
+  private static final long serialVersionUID = -3383272565826389213L;
+
+  public FileAccessForbiddenException() {
+  }
+
+  public FileAccessForbiddenException(String message) {
+    super(message);
+  }
+
+  public FileAccessForbiddenException(Throwable cause) {
+    super(cause);
+  }
+
+  public FileAccessForbiddenException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java
new file mode 100644
index 0000000..236db89
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/FileCloseListener.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+
+public class FileCloseListener implements ChannelFutureListener {
+
+  private FadvisedFileRegion filePart;
+  private String requestUri;
+  private TajoPullServerService pullServerService;
+  private long startTime;
+
+  public FileCloseListener(FadvisedFileRegion filePart,
+                           String requestUri,
+                           long startTime,
+                           TajoPullServerService pullServerService) {
+    this.filePart = filePart;
+    this.requestUri = requestUri;
+    this.pullServerService = pullServerService;
+    this.startTime = startTime;
+  }
+
+  // TODO error handling; distinguish IO/connection failures,
+  //      attribute to appropriate spill output
+  @Override
+  public void operationComplete(ChannelFuture future) {
+    if(future.isSuccess()){
+      filePart.transferSuccessful();
+    }
+    filePart.releaseExternalResources();
+    if (pullServerService != null) {
+      pullServerService.completeFileChunk(filePart, requestUri, startTime);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
new file mode 100644
index 0000000..31db15c
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
@@ -0,0 +1,245 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.pullserver.retriever.DataRetriever;
+import org.apache.tajo.pullserver.retriever.FileChunk;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.*;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedFile;
+import org.jboss.netty.util.CharsetUtil;
+
+import java.io.*;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+public class HttpDataServerHandler extends SimpleChannelUpstreamHandler {
+  private final static Log LOG = LogFactory.getLog(HttpDataServerHandler.class);
+
+  Map<ExecutionBlockId, DataRetriever> retrievers =
+      new ConcurrentHashMap<ExecutionBlockId, DataRetriever>();
+  private String userName;
+  private String appId;
+
+  public HttpDataServerHandler(String userName, String appId) {
+    this.userName= userName;
+    this.appId = appId;
+  }
+
+  @Override
+  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+      throws Exception {
+    HttpRequest request = (HttpRequest) e.getMessage();
+    if (request.getMethod() != GET) {
+      sendError(ctx, METHOD_NOT_ALLOWED);
+      return;
+    }
+
+    String base =
+        ContainerLocalizer.USERCACHE + "/" + userName + "/"
+            + ContainerLocalizer.APPCACHE + "/"
+            + appId + "/output" + "/";
+
+    final Map<String, List<String>> params =
+        new QueryStringDecoder(request.getUri()).getParameters();
+
+    List<FileChunk> chunks = Lists.newArrayList();
+    List<String> taskIds = splitMaps(params.get("ta"));
+    int sid = Integer.valueOf(params.get("sid").get(0));
+    int partitionId = Integer.valueOf(params.get("p").get(0));
+    for (String ta : taskIds) {
+
+      File file = new File(base + "/" + sid + "/" + ta + "/output/" + partitionId);
+      FileChunk chunk = new FileChunk(file, 0, file.length());
+      chunks.add(chunk);
+    }
+
+    FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
+//    try {
+//      file = retriever.handle(ctx, request);
+//    } catch (FileNotFoundException fnf) {
+//      LOG.error(fnf);
+//      sendError(ctx, NOT_FOUND);
+//      return;
+//    } catch (IllegalArgumentException iae) {
+//      LOG.error(iae);
+//      sendError(ctx, BAD_REQUEST);
+//      return;
+//    } catch (FileAccessForbiddenException fafe) {
+//      LOG.error(fafe);
+//      sendError(ctx, FORBIDDEN);
+//      return;
+//    } catch (IOException ioe) {
+//      LOG.error(ioe);
+//      sendError(ctx, INTERNAL_SERVER_ERROR);
+//      return;
+//    }
+
+    // Write the content.
+    Channel ch = e.getChannel();
+    if (file == null) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
+      ch.write(response);
+      if (!isKeepAlive(request)) {
+        ch.close();
+      }
+    }  else {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      long totalSize = 0;
+      for (FileChunk chunk : file) {
+        totalSize += chunk.length();
+      }
+      setContentLength(response, totalSize);
+
+      // Write the initial line and the header.
+      ch.write(response);
+
+      ChannelFuture writeFuture = null;
+
+      for (FileChunk chunk : file) {
+        writeFuture = sendFile(ctx, ch, chunk);
+        if (writeFuture == null) {
+          sendError(ctx, NOT_FOUND);
+          return;
+        }
+      }
+
+      // Decide whether to close the connection or not.
+      if (!isKeepAlive(request)) {
+        // Close the connection when the whole content is written out.
+        writeFuture.addListener(ChannelFutureListener.CLOSE);
+      }
+    }
+  }
+
+  private ChannelFuture sendFile(ChannelHandlerContext ctx,
+                                 Channel ch,
+                                 FileChunk file) throws IOException {
+    RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file.getFile(), "r");
+    } catch (FileNotFoundException fnfe) {
+      return null;
+    }
+
+    ChannelFuture writeFuture;
+    if (ch.getPipeline().get(SslHandler.class) != null) {
+      // Cannot use zero-copy with HTTPS.
+      writeFuture = ch.write(new ChunkedFile(raf, file.startOffset(),
+          file.length(), 8192));
+    } else {
+      // No encryption - use zero-copy.
+      final FileRegion region = new DefaultFileRegion(raf.getChannel(),
+          file.startOffset(), file.length());
+      writeFuture = ch.write(region);
+      writeFuture.addListener(new ChannelFutureListener() {
+        public void operationComplete(ChannelFuture future) {
+          region.releaseExternalResources();
+        }
+      });
+    }
+
+    return writeFuture;
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+      throws Exception {
+    Channel ch = e.getChannel();
+    Throwable cause = e.getCause();
+    if (cause instanceof TooLongFrameException) {
+      sendError(ctx, BAD_REQUEST);
+      return;
+    }
+
+    cause.printStackTrace();
+    if (ch.isConnected()) {
+      sendError(ctx, INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  public static String sanitizeUri(String uri) {
+    // Decode the path.
+    try {
+      uri = URLDecoder.decode(uri, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      try {
+        uri = URLDecoder.decode(uri, "ISO-8859-1");
+      } catch (UnsupportedEncodingException e1) {
+        throw new Error();
+      }
+    }
+
+    // Convert file separators.
+    uri = uri.replace('/', File.separatorChar);
+
+    // Simplistic dumb security check.
+    // You will have to do something serious in the production environment.
+    if (uri.contains(File.separator + ".")
+        || uri.contains("." + File.separator) || uri.startsWith(".")
+        || uri.endsWith(".")) {
+      return null;
+    }
+
+    // Convert to absolute path.
+    return uri;
+  }
+
+  private void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
+    HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+    response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+    response.setContent(ChannelBuffers.copiedBuffer(
+        "Failure: " + status.toString() + "\r\n", CharsetUtil.UTF_8));
+
+    // Close the connection as soon as the error message is sent.
+    ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+  }
+
+  private List<String> splitMaps(List<String> qids) {
+    if (null == qids) {
+      LOG.error("QueryUnitId is EMPTY");
+      return null;
+    }
+
+    final List<String> ret = new ArrayList<String>();
+    for (String qid : qids) {
+      Collections.addAll(ret, qid.split(","));
+    }
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java
new file mode 100644
index 0000000..4c8bd8b
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerPipelineFactory.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.handler.codec.http.HttpContentCompressor;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+
+import static org.jboss.netty.channel.Channels.pipeline;
+
+public class HttpDataServerPipelineFactory implements ChannelPipelineFactory {
+  private String userName;
+  private String appId;
+  public HttpDataServerPipelineFactory(String userName, String appId) {
+    this.userName = userName;
+    this.appId = appId;
+  }
+
+  public ChannelPipeline getPipeline() throws Exception {
+    // Create a default pipeline implementation.
+    ChannelPipeline pipeline = pipeline();
+
+    // Uncomment the following line if you want HTTPS
+    // SSLEngine engine =
+    // SecureChatSslContextFactory.getServerContext().createSSLEngine();
+    // engine.setUseClientMode(false);
+    // pipeline.addLast("ssl", new SslHandler(engine));
+
+    pipeline.addLast("decoder", new HttpRequestDecoder());
+    //pipeline.addLast("aggregator", new HttpChunkAggregator(65536));
+    pipeline.addLast("encoder", new HttpResponseEncoder());
+    pipeline.addLast("chunkedWriter", new ChunkedWriteHandler());
+    pipeline.addLast("deflater", new HttpContentCompressor());
+    pipeline.addLast("handler", new HttpDataServerHandler(userName, appId));
+    return pipeline;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java
new file mode 100644
index 0000000..2cbb101
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/HttpUtil.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import com.google.common.collect.Maps;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URLEncoder;
+import java.util.Map;
+
+public class HttpUtil {
+  public static Map<String,String> getParams(URI uri) throws UnsupportedEncodingException {
+    return getParamsFromQuery(uri.getQuery());
+  }
+
+  /**
+   * It parses a query string into key/value pairs
+   *
+   * @param queryString decoded query string
+   * @return key/value pairs parsed from a given query string
+   * @throws java.io.UnsupportedEncodingException
+   */
+  public static Map<String, String> getParamsFromQuery(String queryString) throws UnsupportedEncodingException {
+    String [] queries = queryString.split("&");
+
+    Map<String,String> params = Maps.newHashMap();
+    String [] param;
+    for (String q : queries) {
+      param = q.split("=");
+      params.put(param[0], param[1]);
+    }
+
+    return params;
+  }
+
+  public static String buildQuery(Map<String,String> params) throws UnsupportedEncodingException {
+    StringBuilder sb = new StringBuilder();
+
+    boolean first = true;
+    for (Map.Entry<String,String> param : params.entrySet()) {
+      if (!first) {
+        sb.append("&");
+      }
+      sb.append(URLEncoder.encode(param.getKey(), "UTF-8")).
+          append("=").
+          append(URLEncoder.encode(param.getValue(), "UTF-8"));
+      first = false;
+    }
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
new file mode 100644
index 0000000..1c63c8a
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
@@ -0,0 +1,654 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
+import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
+import org.apache.hadoop.yarn.server.api.AuxiliaryService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.pullserver.retriever.FileChunk;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.util.TajoIdUtils;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.*;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+public class PullServerAuxService extends AuxiliaryService {
+
+  private static final Log LOG = LogFactory.getLog(PullServerAuxService.class);
+  
+  public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache";
+  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+  public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes";
+  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+  private int port;
+  private ChannelFactory selector;
+  private final ChannelGroup accepted = new DefaultChannelGroup();
+  private HttpPipelineFactory pipelineFact;
+  private int sslFileBufferSize;
+
+  private ApplicationId appId;
+  private QueryId queryId;
+  private FileSystem localFS;
+
+  /**
+   * Should the shuffle use posix_fadvise calls to manage the OS cache during
+   * sendfile
+   */
+  private boolean manageOsCache;
+  private int readaheadLength;
+  private ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
+   
+
+  public static final String PULLSERVER_SERVICEID = "tajo.pullserver";
+
+  private static final Map<String,String> userRsrc =
+    new ConcurrentHashMap<String,String>();
+  private static String userName;
+
+  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+    "tajo.pullserver.ssl.file.buffer.size";
+
+  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+  @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo")
+  static class ShuffleMetrics implements ChannelFutureListener {
+    @Metric({"OutputBytes","PullServer output in bytes"})
+    MutableCounterLong shuffleOutputBytes;
+    @Metric({"Failed","# of failed shuffle outputs"})
+    MutableCounterInt shuffleOutputsFailed;
+    @Metric({"Succeeded","# of succeeded shuffle outputs"})
+    MutableCounterInt shuffleOutputsOK;
+    @Metric({"Connections","# of current shuffle connections"})
+    MutableGaugeInt shuffleConnections;
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if (future.isSuccess()) {
+        shuffleOutputsOK.incr();
+      } else {
+        shuffleOutputsFailed.incr();
+      }
+      shuffleConnections.decr();
+    }
+  }
+
+  final ShuffleMetrics metrics;
+
+  PullServerAuxService(MetricsSystem ms) {
+    super("httpshuffle");
+    metrics = ms.register(new ShuffleMetrics());
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public PullServerAuxService() {
+    this(DefaultMetricsSystem.instance());
+  }
+
+  /**
+   * Serialize the shuffle port into a ByteBuffer for use later on.
+   * @param port the port to be sent to the ApplciationMaster
+   * @return the serialized form of the port.
+   */
+  public static ByteBuffer serializeMetaData(int port) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer port_dob = new DataOutputBuffer();
+    port_dob.writeInt(port);
+    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+  }
+
+  /**
+   * A helper function to deserialize the metadata returned by PullServerAuxService.
+   * @param meta the metadata returned by the PullServerAuxService
+   * @return the port the PullServer Handler is listening on to serve shuffle data.
+   */
+  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+    //TODO this should be returning a class not just an int
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    return in.readInt();
+  }
+
+  @Override
+  public void initializeApplication(ApplicationInitializationContext appInitContext) {
+    // TODO these bytes should be versioned
+    // TODO: Once SHuffle is out of NM, this can use MR APIs
+    this.appId = appInitContext.getApplicationId();
+    this.queryId = TajoIdUtils.parseQueryId(appId.toString());
+    this.userName = appInitContext.getUser();
+    userRsrc.put(this.appId.toString(), this.userName);
+  }
+
+  @Override
+  public void stopApplication(ApplicationTerminationContext appStopContext) {
+    userRsrc.remove(appStopContext.getApplicationId().toString());
+  }
+
+  @Override
+  public synchronized void init(Configuration conf) {
+    try {
+      manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
+          DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
+
+      readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
+          DEFAULT_SHUFFLE_READAHEAD_BYTES);
+
+      ThreadFactory bossFactory = new ThreadFactoryBuilder()
+          .setNameFormat("PullServerAuxService Netty Boss #%d")
+          .build();
+      ThreadFactory workerFactory = new ThreadFactoryBuilder()
+          .setNameFormat("PullServerAuxService Netty Worker #%d")
+          .build();
+
+      selector = new NioServerSocketChannelFactory(
+          Executors.newCachedThreadPool(bossFactory),
+          Executors.newCachedThreadPool(workerFactory));
+
+      localFS = new LocalFileSystem();
+      super.init(new Configuration(conf));
+    } catch (Throwable t) {
+      LOG.error(t);
+    }
+  }
+
+  // TODO change AbstractService to throw InterruptedException
+  @Override
+  public synchronized void start() {
+    Configuration conf = getConfig();
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    try {
+      pipelineFact = new HttpPipelineFactory(conf);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    bootstrap.setPipelineFactory(pipelineFact);
+    port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
+        ConfVars.PULLSERVER_PORT.defaultIntVal);
+    Channel ch = bootstrap.bind(new InetSocketAddress(port));
+    accepted.add(ch);
+    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    conf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port));
+    pipelineFact.PullServer.setPort(port);
+    LOG.info(getName() + " listening on port " + port);
+    super.start();
+
+    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  @Override
+  public synchronized void stop() {
+    try {
+      accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+      ServerBootstrap bootstrap = new ServerBootstrap(selector);
+      bootstrap.releaseExternalResources();
+      pipelineFact.destroy();
+
+      localFS.close();
+    } catch (Throwable t) {
+      LOG.error(t);
+    } finally {
+      super.stop();
+    }
+  }
+
+  @Override
+  public synchronized ByteBuffer getMetaData() {
+    try {
+      return serializeMetaData(port); 
+    } catch (IOException e) {
+      LOG.error("Error during getMeta", e);
+      // TODO add API to AuxiliaryServices to report failures
+      return null;
+    }
+  }
+
+  class HttpPipelineFactory implements ChannelPipelineFactory {
+
+    final PullServer PullServer;
+    private SSLFactory sslFactory;
+
+    public HttpPipelineFactory(Configuration conf) throws Exception {
+      PullServer = new PullServer(conf);
+      if (conf.getBoolean(ConfVars.SHUFFLE_SSL_ENABLED_KEY.varname,
+          ConfVars.SHUFFLE_SSL_ENABLED_KEY.defaultBoolVal)) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+        sslFactory.init();
+      }
+    }
+
+    public void destroy() {
+      if (sslFactory != null) {
+        sslFactory.destroy();
+      }
+    }
+
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+      ChannelPipeline pipeline = Channels.pipeline();
+      if (sslFactory != null) {
+        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+      }
+      pipeline.addLast("decoder", new HttpRequestDecoder());
+      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+      pipeline.addLast("encoder", new HttpResponseEncoder());
+      pipeline.addLast("chunking", new ChunkedWriteHandler());
+      pipeline.addLast("shuffle", PullServer);
+      return pipeline;
+      // TODO factor security manager into pipeline
+      // TODO factor out encode/decode to permit binary shuffle
+      // TODO factor out decode of index to permit alt. models
+    }
+  }
+
+  class PullServer extends SimpleChannelUpstreamHandler {
+    private final Configuration conf;
+    private final LocalDirAllocator lDirAlloc = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
+    private int port;
+
+    public PullServer(Configuration conf) {
+      this.conf = conf;
+      this.port = conf.getInt(ConfVars.PULLSERVER_PORT.varname, ConfVars.PULLSERVER_PORT.defaultIntVal);
+    }
+    
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    private List<String> splitMaps(List<String> mapq) {
+      if (null == mapq) {
+        return null;
+      }
+      final List<String> ret = new ArrayList<String>();
+      for (String s : mapq) {
+        Collections.addAll(ret, s.split(","));
+      }
+      return ret;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+        throws Exception {
+
+      HttpRequest request = (HttpRequest) e.getMessage();
+      if (request.getMethod() != GET) {
+        sendError(ctx, METHOD_NOT_ALLOWED);
+        return;
+      }
+
+      // Parsing the URL into key-values
+      final Map<String, List<String>> params =
+          new QueryStringDecoder(request.getUri()).getParameters();
+      final List<String> types = params.get("type");
+      final List<String> taskIdList = params.get("ta");
+      final List<String> subQueryIds = params.get("sid");
+      final List<String> partitionIds = params.get("p");
+
+      if (types == null || taskIdList == null || subQueryIds == null
+          || partitionIds == null) {
+        sendError(ctx, "Required type, taskIds, subquery Id, and partition id",
+            BAD_REQUEST);
+        return;
+      }
+
+      if (types.size() != 1 || subQueryIds.size() != 1) {
+        sendError(ctx, "Required type, taskIds, subquery Id, and partition id",
+            BAD_REQUEST);
+        return;
+      }
+
+      final List<FileChunk> chunks = Lists.newArrayList();
+
+      String repartitionType = types.get(0);
+      String sid = subQueryIds.get(0);
+      String partitionId = partitionIds.get(0);
+      List<String> taskIds = splitMaps(taskIdList);
+
+      // the working dir of tajo worker for each query
+      String queryBaseDir = queryId + "/output" + "/";
+
+      LOG.info("PullServer request param: repartitionType=" + repartitionType +
+          ", sid=" + sid + ", partitionId=" + partitionId + ", taskIds=" + taskIdList);
+
+      String taskLocalDir = conf.get(ConfVars.WORKER_TEMPORAL_DIR.varname);
+      if (taskLocalDir == null ||
+          taskLocalDir.equals("")) {
+        LOG.error("Tajo local directory should be specified.");
+      }
+      LOG.info("PullServer baseDir: " + taskLocalDir + "/" + queryBaseDir);
+
+      // if a subquery requires a range partitioning
+      if (repartitionType.equals("r")) {
+        String ta = taskIds.get(0);
+        Path path = localFS.makeQualified(
+            lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/"
+                + ta + "/output/", conf));
+
+        String startKey = params.get("start").get(0);
+        String endKey = params.get("end").get(0);
+        boolean last = params.get("final") != null;
+
+        FileChunk chunk;
+        try {
+          chunk = getFileCunks(path, startKey, endKey, last);
+        } catch (Throwable t) {
+          LOG.error("ERROR Request: " + request.getUri(), t);
+          sendError(ctx, "Cannot get file chunks to be sent", BAD_REQUEST);
+          return;
+        }
+        if (chunk != null) {
+          chunks.add(chunk);
+        }
+
+        // if a subquery requires a hash repartition  or a scattered hash repartition
+      } else if (repartitionType.equals("h") || repartitionType.equals("s")) {
+        for (String ta : taskIds) {
+          Path path = localFS.makeQualified(
+              lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" +
+                  ta + "/output/" + partitionId, conf));
+          File file = new File(path.toUri());
+          FileChunk chunk = new FileChunk(file, 0, file.length());
+          chunks.add(chunk);
+        }
+      } else {
+        LOG.error("Unknown repartition type: " + repartitionType);
+        return;
+      }
+
+      // Write the content.
+      Channel ch = e.getChannel();
+      if (chunks.size() == 0) {
+        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
+        ch.write(response);
+        if (!isKeepAlive(request)) {
+          ch.close();
+        }
+      }  else {
+        FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
+        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+        long totalSize = 0;
+        for (FileChunk chunk : file) {
+          totalSize += chunk.length();
+        }
+        setContentLength(response, totalSize);
+
+        // Write the initial line and the header.
+        ch.write(response);
+
+        ChannelFuture writeFuture = null;
+
+        for (FileChunk chunk : file) {
+          writeFuture = sendFile(ctx, ch, chunk);
+          if (writeFuture == null) {
+            sendError(ctx, NOT_FOUND);
+            return;
+          }
+        }
+
+        // Decide whether to close the connection or not.
+        if (!isKeepAlive(request)) {
+          // Close the connection when the whole content is written out.
+          writeFuture.addListener(ChannelFutureListener.CLOSE);
+        }
+      }
+    }
+
+    private ChannelFuture sendFile(ChannelHandlerContext ctx,
+                                   Channel ch,
+                                   FileChunk file) throws IOException {
+      RandomAccessFile spill;
+      try {
+        spill = new RandomAccessFile(file.getFile(), "r");
+      } catch (FileNotFoundException e) {
+        LOG.info(file.getFile() + " not found");
+        return null;
+      }
+      ChannelFuture writeFuture;
+      if (ch.getPipeline().get(SslHandler.class) == null) {
+        final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
+            file.startOffset(), file.length(), manageOsCache, readaheadLength,
+            readaheadPool, file.getFile().getAbsolutePath());
+        writeFuture = ch.write(partition);
+        writeFuture.addListener(new FileCloseListener(partition, null, 0, null));
+      } else {
+        // HTTPS cannot be done with zero copy.
+        final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
+            file.startOffset(), file.length(), sslFileBufferSize,
+            manageOsCache, readaheadLength, readaheadPool,
+            file.getFile().getAbsolutePath());
+        writeFuture = ch.write(chunk);
+      }
+      metrics.shuffleConnections.incr();
+      metrics.shuffleOutputBytes.incr(file.length()); // optimistic
+      return writeFuture;
+    }
+
+    private void sendError(ChannelHandlerContext ctx,
+        HttpResponseStatus status) {
+      sendError(ctx, "", status);
+    }
+
+    private void sendError(ChannelHandlerContext ctx, String message,
+        HttpResponseStatus status) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+      response.setContent(
+        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+
+      // Close the connection as soon as the error message is sent.
+      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+        throws Exception {
+      Channel ch = e.getChannel();
+      Throwable cause = e.getCause();
+      if (cause instanceof TooLongFrameException) {
+        sendError(ctx, BAD_REQUEST);
+        return;
+      }
+
+      LOG.error("PullServer error: ", cause);
+      if (ch.isConnected()) {
+        LOG.error("PullServer error " + e);
+        sendError(ctx, INTERNAL_SERVER_ERROR);
+      }
+    }
+  }
+
+  public FileChunk getFileCunks(Path outDir,
+                                      String startKey,
+                                      String endKey,
+                                      boolean last) throws IOException {
+    BSTIndex index = new BSTIndex(new TajoConf());
+    BSTIndex.BSTIndexReader idxReader =
+        index.getIndexReader(new Path(outDir, "index"));
+    idxReader.open();
+    Schema keySchema = idxReader.getKeySchema();
+    TupleComparator comparator = idxReader.getComparator();
+
+    LOG.info("BSTIndex is loaded from disk (" + idxReader.getFirstKey() + ", "
+        + idxReader.getLastKey());
+
+    File data = new File(URI.create(outDir.toUri() + "/output"));
+    byte [] startBytes = Base64.decodeBase64(startKey);
+    byte [] endBytes = Base64.decodeBase64(endKey);
+
+    RowStoreDecoder decoder = RowStoreUtil.createDecoder(keySchema);
+    Tuple start;
+    Tuple end;
+    try {
+      start = decoder.toTuple(startBytes);
+    } catch (Throwable t) {
+      throw new IllegalArgumentException("StartKey: " + startKey
+          + ", decoded byte size: " + startBytes.length, t);
+    }
+
+    try {
+      end = decoder.toTuple(endBytes);
+    } catch (Throwable t) {
+      throw new IllegalArgumentException("EndKey: " + endKey
+          + ", decoded byte size: " + endBytes.length, t);
+    }
+
+
+    if(!comparator.isAscendingFirstKey()) {
+      Tuple tmpKey = start;
+      start = end;
+      end = tmpKey;
+    }
+
+    LOG.info("GET Request for " + data.getAbsolutePath() + " (start="+start+", end="+ end +
+        (last ? ", last=true" : "") + ")");
+
+    if (idxReader.getFirstKey() == null && idxReader.getLastKey() == null) { // if # of rows is zero
+      LOG.info("There is no contents");
+      return null;
+    }
+
+    if (comparator.compare(end, idxReader.getFirstKey()) < 0 ||
+        comparator.compare(idxReader.getLastKey(), start) < 0) {
+      LOG.info("Out of Scope (indexed data [" + idxReader.getFirstKey() + ", " + idxReader.getLastKey() +
+          "], but request start:" + start + ", end: " + end);
+      return null;
+    }
+
+    long startOffset;
+    long endOffset;
+    try {
+      startOffset = idxReader.find(start);
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + "[" + start + ", " + end +")" + ", idx min: "
+          + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+    try {
+      endOffset = idxReader.find(end);
+      if (endOffset == -1) {
+        endOffset = idxReader.find(end, true);
+      }
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + "[" + start + ", " + end +")" + ", idx min: "
+          + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+
+    // if startOffset == -1 then case 2-1 or case 3
+    if (startOffset == -1) { // this is a hack
+      // if case 2-1 or case 3
+      try {
+        startOffset = idxReader.find(start, true);
+      } catch (IOException ioe) {
+        LOG.error("State Dump (the requested range: "
+            + "[" + start + ", " + end +")" + ", idx min: "
+            + idxReader.getFirstKey() + ", idx max: "
+            + idxReader.getLastKey());
+        throw ioe;
+      }
+    }
+
+    if (startOffset == -1) {
+      throw new IllegalStateException("startOffset " + startOffset + " is negative \n" +
+          "State Dump (the requested range: "
+          + "[" + start + ", " + end +")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+    }
+
+    // if greater than indexed values
+    if (last || (endOffset == -1
+        && comparator.compare(idxReader.getLastKey(), end) < 0)) {
+      endOffset = data.length();
+    }
+
+    FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset);
+    LOG.info("Retrieve File Chunk: " + chunk);
+    return chunk;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java
new file mode 100644
index 0000000..564950f
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import org.apache.commons.lang.reflect.MethodUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.nativeio.NativeIO;
+
+import java.io.FileDescriptor;
+import java.lang.reflect.Method;
+
+public class PullServerUtil {
+  private static final Log LOG = LogFactory.getLog(PullServerUtil.class);
+
+  private static boolean nativeIOPossible = false;
+  private static Method posixFadviseIfPossible;
+
+  static {
+    if (NativeIO.isAvailable() && loadNativeIO()) {
+      nativeIOPossible = true;
+    } else {
+      LOG.warn("Unable to load hadoop nativeIO");
+    }
+  }
+
+  public static boolean isNativeIOPossible() {
+    return nativeIOPossible;
+  }
+
+  /**
+   * Call posix_fadvise on the given file descriptor. See the manpage
+   * for this syscall for more information. On systems where this
+   * call is not available, does nothing.
+   */
+  public static void posixFadviseIfPossible(String identifier, java.io.FileDescriptor fd,
+                                            long offset, long len, int flags) {
+    if (nativeIOPossible) {
+      try {
+        posixFadviseIfPossible.invoke(null, identifier, fd, offset, len, flags);
+      } catch (Throwable t) {
+        nativeIOPossible = false;
+        LOG.warn("Failed to manage OS cache for " + identifier, t);
+      }
+    }
+  }
+
+  /* load hadoop native method if possible */
+  private static boolean loadNativeIO() {
+    boolean loaded = true;
+    if (nativeIOPossible) return loaded;
+
+    Class[] parameters = {String.class, FileDescriptor.class, Long.TYPE, Long.TYPE, Integer.TYPE};
+    try {
+      Method getCacheManipulator = MethodUtils.getAccessibleMethod(NativeIO.POSIX.class, "getCacheManipulator", new Class[0]);
+      Class posixClass;
+      if (getCacheManipulator != null) {
+        Object posix = MethodUtils.invokeStaticMethod(NativeIO.POSIX.class, "getCacheManipulator", null);
+        posixClass = posix.getClass();
+      } else {
+        posixClass = NativeIO.POSIX.class;
+      }
+      posixFadviseIfPossible = MethodUtils.getAccessibleMethod(posixClass, "posixFadviseIfPossible", parameters);
+    } catch (Throwable e) {
+      loaded = false;
+      LOG.warn("Failed to access posixFadviseIfPossible :" + e.getMessage());
+    }
+
+    if (posixFadviseIfPossible == null) {
+      loaded = false;
+    }
+    return loaded;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java
----------------------------------------------------------------------
diff --git a/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java
new file mode 100644
index 0000000..d030eed
--- /dev/null
+++ b/tajo-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import 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.CompositeService;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.pullserver.PullServerAuxService.PullServer;
+import org.apache.tajo.util.StringUtils;
+
+public class TajoPullServer extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(TajoPullServer.class);
+
+  private TajoPullServerService pullService;
+  private TajoConf systemConf;
+
+  public TajoPullServer() {
+    super(TajoPullServer.class.getName());
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.systemConf = (TajoConf)conf;
+    pullService = new TajoPullServerService();
+    addService(pullService);
+
+    super.init(conf);
+  }
+
+  public void startPullServer(TajoConf systemConf) {
+    init(systemConf);
+    start();
+  }
+
+  public void start() {
+    super.start();
+
+  }
+
+  public static void main(String[] args) throws Exception {
+    StringUtils.startupShutdownMessage(PullServer.class, args, LOG);
+
+    if (!TajoPullServerService.isStandalone()) {
+      LOG.fatal("TAJO_PULLSERVER_STANDALONE env variable is not 'true'");
+      return;
+    }
+
+    TajoConf tajoConf = new TajoConf();
+    tajoConf.addResource(new Path(TajoConstants.SYSTEM_CONF_FILENAME));
+
+    (new TajoPullServer()).startPullServer(tajoConf);
+  }
+}


[04/10] tajo git commit: TAJO-1234: Rearrange timezone in date/time types.

Posted by ji...@apache.org.
TAJO-1234: Rearrange timezone in date/time types.

Closes #290


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

Branch: refs/heads/index_support
Commit: facd1ddcc85a088ab8ba4e42dce2c99f6fe7e5de
Parents: 95cf4b9
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 8 16:43:53 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 8 16:43:53 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/client/QueryClient.java     |   2 +
 .../org/apache/tajo/client/QueryClientImpl.java |   8 +-
 .../apache/tajo/client/SessionConnection.java   |  33 ++++
 .../org/apache/tajo/client/TajoClientUtil.java  |   3 +-
 .../org/apache/tajo/jdbc/FetchResultSet.java    |   1 +
 .../apache/tajo/jdbc/TajoMemoryResultSet.java   |   5 +-
 .../org/apache/tajo/jdbc/TajoResultSet.java     |   2 +
 .../org/apache/tajo/jdbc/TajoResultSetBase.java |  38 +++-
 .../java/org/apache/tajo/OverridableConf.java   |   7 +-
 .../main/java/org/apache/tajo/SessionVars.java  |   4 +-
 .../java/org/apache/tajo/TajoConstants.java     |   2 +
 .../java/org/apache/tajo/conf/TajoConf.java     |  25 +--
 .../main/java/org/apache/tajo/datum/Datum.java  |   2 +-
 .../org/apache/tajo/datum/DatumFactory.java     |  57 ++++--
 .../tajo/json/TimeZoneGsonSerdeAdapter.java     |  40 ++++
 .../apache/tajo/storage/StorageConstants.java   |  22 ++-
 .../apache/tajo/util/datetime/DateTimeUtil.java |  23 ++-
 .../apache/tajo/datum/TestTimestampDatum.java   |   7 +-
 .../tajo/engine/function/builtin/AvgDouble.java |   3 -
 .../engine/function/datetime/CurrentDate.java   |  20 +-
 .../function/datetime/DatePartFromTime.java     |  20 +-
 .../datetime/DatePartFromTimestamp.java         |  18 +-
 .../function/datetime/ToCharTimestamp.java      |  18 +-
 .../function/datetime/ToTimestampText.java      |  15 +-
 .../tajo/engine/function/string/BTrim.java      |   4 +-
 .../tajo/engine/function/string/LTrim.java      |   4 +-
 .../tajo/engine/function/string/Lpad.java       |   4 +-
 .../tajo/engine/function/string/RTrim.java      |   4 +-
 .../engine/function/string/RegexpReplace.java   |   4 +-
 .../tajo/engine/function/string/Rpad.java       |   4 +-
 .../apache/tajo/engine/json/CoreGsonHelper.java |   2 +
 .../apache/tajo/engine/query/QueryContext.java  |   4 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |   8 +
 .../org/apache/tajo/TajoTestingCluster.java     |   3 +
 .../apache/tajo/engine/eval/ExprTestBase.java   |  59 +++---
 .../tajo/engine/eval/TestIntervalType.java      |  46 ++---
 .../tajo/engine/eval/TestSQLDateTimeTypes.java  |  18 +-
 .../tajo/engine/eval/TestSQLExpression.java     |  67 ++++---
 .../function/TestConditionalExpressions.java    |  16 +-
 .../engine/function/TestDateTimeFunctions.java  | 186 ++++++++++++-------
 .../tajo/engine/query/TestSelectQuery.java      |  50 ++++-
 .../apache/tajo/engine/query/TestSortQuery.java |  23 +--
 .../org/apache/tajo/jdbc/TestResultSet.java     |  13 +-
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java |   6 -
 .../TestSelectQuery/timezoned/table1.tbl        |   3 +
 .../TestSelectQuery/datetime_table_ddl.sql      |   4 +
 .../datetime_table_timezoned_ddl.sql            |   4 +
 .../TestSelectQuery/testTimezonedTable1.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable2.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable3.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable1.result  |   5 +
 .../TestSelectQuery/testTimezonedTable2.result  |   5 +
 .../TestSelectQuery/testTimezonedTable3.result  |   5 +
 tajo-docs/src/main/sphinx/table_management.rst  |   5 +-
 .../src/main/sphinx/table_management/csv.rst    |   4 +-
 .../sphinx/table_management/table_overview.rst  |  98 ++++++++++
 .../apache/tajo/jdbc/TajoMetaDataResultSet.java |   2 +
 .../org/apache/tajo/plan/ExprAnnotator.java     |  96 ++++++----
 .../org/apache/tajo/plan/LogicalPlanner.java    |   8 +
 .../org/apache/tajo/plan/expr/CastEval.java     |  28 ++-
 .../tajo/plan/expr/GeneralFunctionEval.java     |   8 +-
 .../tajo/plan/function/GeneralFunction.java     |   4 +-
 .../plan/serder/EvalTreeProtoDeserializer.java  |   7 +-
 .../plan/serder/EvalTreeProtoSerializer.java    |   3 +
 .../apache/tajo/plan/serder/PlanGsonHelper.java |   2 +
 .../org/apache/tajo/plan/util/PlannerUtil.java  |   4 +
 tajo-plan/src/main/proto/Plan.proto             |   1 +
 .../storage/TextSerializerDeserializer.java     |   7 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |   2 +-
 .../tajo/storage/text/CSVLineSerializer.java    |   2 +-
 .../text/TextFieldSerializerDeserializer.java   |  46 ++++-
 72 files changed, 902 insertions(+), 356 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index a59e107..acc72b9 100644
--- a/CHANGES
+++ b/CHANGES
@@ -85,6 +85,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1234: Rearrange timezone in date/time types. (hyunsik)
+
     TAJO-1231: Implicit table properties in session are not stored in 
     table property. (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java b/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
index 9b24663..32ef97d 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
@@ -42,6 +42,8 @@ public interface QueryClient extends Closeable {
 
   public SessionIdProto getSessionId();
 
+  public Map<String, String> getClientSideSessionVars();
+
   public String getBaseDatabase();
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
index 5b78959..1cee515 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
@@ -73,6 +73,11 @@ public class QueryClientImpl implements QueryClient {
   }
 
   @Override
+  public Map<String, String> getClientSideSessionVars() {
+    return connection.getClientSideSessionVars();
+  }
+
+  @Override
   public String getBaseDatabase() {
     return connection.getBaseDatabase();
   }
@@ -455,7 +460,8 @@ public class QueryClientImpl implements QueryClient {
       return new TajoMemoryResultSet(
           new Schema(serializedResultSet.getSchema()),
           serializedResultSet.getSerializedTuplesList(),
-          serializedResultSet.getSerializedTuplesCount());
+          serializedResultSet.getSerializedTuplesCount(),
+          getClientSideSessionVars());
     } catch (Exception e) {
       throw new ServiceException(e.getMessage(), e);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
index 42085a2..922984f 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.conf.TajoConf;
@@ -39,6 +40,7 @@ import org.jboss.netty.channel.ConnectTimeoutException;
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -68,6 +70,9 @@ public class SessionConnection implements Closeable {
 
   private AtomicBoolean closed = new AtomicBoolean(false);
 
+  /** session variable cache */
+  Map<String, String> clientSideSessionVars = new ConcurrentHashMap<String, String>();
+
 
   public SessionConnection(TajoConf conf) throws IOException {
     this(conf, NetUtils.createSocketAddr(conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS)), null);
@@ -105,6 +110,10 @@ public class SessionConnection implements Closeable {
     this.baseDatabase = baseDatabase != null ? baseDatabase : null;
   }
 
+  public Map<String, String> getClientSideSessionVars() {
+    return Collections.unmodifiableMap(clientSideSessionVars);
+  }
+
   public <T> T getStub(QueryId queryId, Class protocolClass, boolean asyncMode) throws NoSuchMethodException,
       ConnectTimeoutException, ClassNotFoundException {
     InetSocketAddress addr = queryMasterMap.get(queryId);
@@ -177,6 +186,17 @@ public class SessionConnection implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
 
+        // keep client-side session variables
+        for (Map.Entry<String, String> entry : variables.entrySet()) {
+          String key = entry.getKey();
+          if (SessionVars.exists(entry.getKey())) {
+            SessionVars configKey = SessionVars.get(key);
+            if (configKey.getMode() == SessionVars.VariableMode.CLI_SIDE_VAR) {
+              clientSideSessionVars.put(key, entry.getValue());
+            }
+          }
+        }
+
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
         KeyValueSet keyValueSet = new KeyValueSet();
         keyValueSet.putAll(variables);
@@ -195,6 +215,13 @@ public class SessionConnection implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
 
+        // Remove matched session vars
+        for (String key : variables) {
+          if (clientSideSessionVars.containsKey(key)) {
+            clientSideSessionVars.remove(key);
+          }
+        }
+
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
         ClientProtos.UpdateSessionVariableRequest request = ClientProtos.UpdateSessionVariableRequest.newBuilder()
             .setSessionId(sessionId)
@@ -208,6 +235,12 @@ public class SessionConnection implements Closeable {
     return new ServerCallable<String>(connPool, getTajoMasterAddr(), TajoMasterClientProtocol.class, false, true) {
 
       public String call(NettyClientBase client) throws ServiceException {
+
+        // If a desired variable is client side one and exists in the cache, immediately return the variable.
+        if (clientSideSessionVars.containsKey(varname)) {
+          return clientSideSessionVars.get(varname);
+        }
+
         checkSessionAndGet(client);
 
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
index 7aed335..bf9d111 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
@@ -88,7 +88,8 @@ public class TajoClientUtil {
       return new TajoMemoryResultSet(
           new Schema(serializedResultSet.getSchema()),
           serializedResultSet.getSerializedTuplesList(),
-          response.getMaxRowNum());
+          response.getMaxRowNum(),
+          client.getClientSideSessionVars());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
index 78674b1..18b7c1a 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
@@ -35,6 +35,7 @@ public class FetchResultSet extends TajoResultSetBase {
   private boolean finished = false;
 
   public FetchResultSet(QueryClient tajoClient, Schema schema, QueryId queryId, int fetchRowNum) {
+    super(tajoClient.getClientSideSessionVars());
     this.tajoClient = tajoClient;
     this.queryId = queryId;
     this.fetchRowNum = fetchRowNum;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
index 84fafda..d0898f5 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
@@ -26,6 +26,7 @@ import org.apache.tajo.storage.Tuple;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 public class TajoMemoryResultSet extends TajoResultSetBase {
@@ -33,7 +34,9 @@ public class TajoMemoryResultSet extends TajoResultSetBase {
   private AtomicBoolean closed = new AtomicBoolean(false);
   private RowStoreUtil.RowStoreDecoder decoder;
 
-  public TajoMemoryResultSet(Schema schema, List<ByteString> serializedTuples, int maxRowNum) {
+  public TajoMemoryResultSet(Schema schema, List<ByteString> serializedTuples, int maxRowNum,
+                             Map<String, String> clientSideSessionVars) {
+    super(clientSideSessionVars);
     this.schema = schema;
     this.totalRow = maxRowNum;
     this.serializedTuples = serializedTuples;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
index e99c0ff..e2ac3b1 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
@@ -56,12 +56,14 @@ public class TajoResultSet extends TajoResultSetBase {
   private AtomicBoolean closed = new AtomicBoolean(false);
 
   public TajoResultSet(QueryClient tajoClient, QueryId queryId) {
+    super(tajoClient.getClientSideSessionVars());
     this.tajoClient = tajoClient;
     this.queryId = queryId;
     init();
   }
 
   public TajoResultSet(QueryClient tajoClient, QueryId queryId, TajoConf conf, TableDesc table) throws IOException {
+    super(tajoClient.getClientSideSessionVars());
     this.tajoClient = tajoClient;
     this.queryId = queryId;
     this.conf = conf;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
index 78d8bde..051ea75 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
@@ -18,14 +18,16 @@
 
 package org.apache.tajo.jdbc;
 
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.*;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
@@ -37,12 +39,30 @@ import java.util.Map;
 import java.util.TimeZone;
 
 public abstract class TajoResultSetBase implements ResultSet {
+  protected final Map<String, String> clientSideSessionVars;
+  protected TimeZone timezone;
+
   protected int curRow;
   protected long totalRow;
   protected boolean wasNull;
   protected Schema schema;
   protected Tuple cur;
 
+  public TajoResultSetBase(@Nullable Map<String, String> clientSideSessionVars) {
+    this.clientSideSessionVars = clientSideSessionVars;
+
+    if (clientSideSessionVars != null) {
+
+      if (clientSideSessionVars.containsKey(SessionVars.TZ.name())) {
+        String timezoneId = clientSideSessionVars.get(SessionVars.TZ.name());
+        this.timezone = TimeZone.getTimeZone(timezoneId);
+      } else {
+        this.timezone = TimeZone.getTimeZone(TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+      }
+
+    }
+  }
+
   protected void init() {
     cur = null;
     curRow = 0;
@@ -226,13 +246,13 @@ public abstract class TajoResultSetBase implements ResultSet {
       case FLOAT8:  return d.asFloat8();
       case NUMERIC:  return d.asFloat8();
       case DATE: {
-        return getDate((DateDatum)d, TajoConf.getCurrentTimeZone());
+        return getDate((DateDatum)d, timezone);
       }
       case TIME: {
-        return getTime((TimeDatum)d, TajoConf.getCurrentTimeZone());
+        return getTime((TimeDatum)d, timezone);
       }
       case TIMESTAMP: {
-        return getTimestamp((TimestampDatum) d, TajoConf.getCurrentTimeZone());
+        return getTimestamp((TimestampDatum) d, timezone);
       }
       default: return d.asChars();
     }
@@ -289,10 +309,10 @@ public abstract class TajoResultSetBase implements ResultSet {
       case BOOLEAN:
         return String.valueOf(datum.asBool());
       case TIME: {
-        return ((TimeDatum)datum).asChars(TajoConf.getCurrentTimeZone(), false);
+        return ((TimeDatum)datum).asChars(timezone, false);
       }
       case TIMESTAMP: {
-        return ((TimestampDatum)datum).asChars(TajoConf.getCurrentTimeZone(), false);
+        return ((TimestampDatum)datum).asChars(timezone, false);
       }
       default :
         return datum.asChars();
@@ -307,7 +327,7 @@ public abstract class TajoResultSetBase implements ResultSet {
       return null;
     }
 
-    return getDate((DateDatum)datum, TajoConf.getCurrentTimeZone());
+    return getDate((DateDatum)datum, null);
   }
 
   @Override
@@ -347,7 +367,7 @@ public abstract class TajoResultSetBase implements ResultSet {
       return null;
     }
 
-    return getTime((TimeDatum)datum, TajoConf.getCurrentTimeZone());
+    return getTime((TimeDatum)datum, timezone);
 
   }
 
@@ -388,7 +408,7 @@ public abstract class TajoResultSetBase implements ResultSet {
       return null;
     }
 
-    return getTimestamp((TimestampDatum)datum, TajoConf.getCurrentTimeZone());
+    return getTimestamp((TimestampDatum)datum, timezone);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java b/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
index 84be00e..b7a5da7 100644
--- a/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
@@ -181,10 +181,13 @@ public class OverridableConf extends KeyValueSet {
   private void assertRegisteredEnum(ConfigKey key) {
     boolean registered = false;
 
-    for (ConfigType c : configTypes) {
-      registered = key.type() == c;
+    if (configTypes != null) {
+      for (ConfigType c : configTypes) {
+        registered = key.type() == c;
+      }
     }
 
+    // default permitted keys
     registered |= key.type() == ConfigType.SESSION || key.type() != ConfigType.SYSTEM;
 
     Preconditions.checkArgument(registered, key.keyname() + " (" + key.type() + ") is not allowed in " +

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
----------------------------------------------------------------------
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 dd33e12..fa717d1 100644
--- a/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
+++ b/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
@@ -71,8 +71,8 @@ public enum SessionVars implements ConfigKey {
   ON_ERROR_STOP(ConfVars.$CLI_ERROR_STOP, "tsql will exist if an error occurs.", CLI_SIDE_VAR),
 
   // Timezone & Date ----------------------------------------------------------
-  TZ(ConfVars.$TIMEZONE, "Sets timezone", FROM_SHELL_ENV),
-  DATE_ORDER(ConfVars.$DATE_ORDER, "date order (default is YMD)", FROM_SHELL_ENV),
+  TZ(ConfVars.$TIMEZONE, "Sets timezone", CLI_SIDE_VAR),
+  DATE_ORDER(ConfVars.$DATE_ORDER, "date order (default is YMD)", CLI_SIDE_VAR),
 
   // Locales and Character set ------------------------------------------------
   // TODO - they are reserved variables, and we should support them.

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
index 1cc28af..9250edd 100644
--- a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
@@ -31,6 +31,8 @@ public class TajoConstants {
   public static final String DEFAULT_DATABASE_NAME = "default";
   public static final String DEFAULT_SCHEMA_NAME = "public";
 
+  /** Java 6 only recognize GMT instead of UTC. So, we should keep using GMT. */
+  public static final String DEFAULT_SYSTEM_TIMEZONE = "GMT";
 
   public static final String EMPTY_STRING = "";
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index 312abfb..b00b322 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
@@ -44,7 +44,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 public class TajoConf extends Configuration {
 
-  private static TimeZone CURRENT_TIMEZONE;
+  private static TimeZone SYSTEM_TIMEZONE;
   private static int DATE_ORDER = -1;
   private static final ReentrantReadWriteLock confLock = new ReentrantReadWriteLock();
   private static final Lock writeLock = confLock.writeLock();
@@ -83,29 +83,18 @@ public class TajoConf extends Configuration {
   }
 
   private static void confStaticInit() {
-    TimeZone.setDefault(getCurrentTimeZone());
+    TimeZone.setDefault(getSystemTimezone());
     getDateOrder();
   }
 
-  public static TimeZone getCurrentTimeZone() {
+  public static TimeZone getSystemTimezone() {
     writeLock.lock();
     try {
-      if (CURRENT_TIMEZONE == null) {
+      if (SYSTEM_TIMEZONE == null) {
         TajoConf tajoConf = new TajoConf();
-        CURRENT_TIMEZONE = TimeZone.getTimeZone(tajoConf.getVar(ConfVars.$TIMEZONE));
+        SYSTEM_TIMEZONE = TimeZone.getTimeZone(tajoConf.getVar(ConfVars.$TIMEZONE));
       }
-      return CURRENT_TIMEZONE;
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
-  public static TimeZone setCurrentTimeZone(TimeZone timeZone) {
-    writeLock.lock();
-    try {
-      TimeZone oldTimeZone = CURRENT_TIMEZONE;
-      CURRENT_TIMEZONE = timeZone;
-      return oldTimeZone;
+      return SYSTEM_TIMEZONE;
     } finally {
       writeLock.unlock();
     }
@@ -360,7 +349,7 @@ public class TajoConf extends Configuration {
     $CLI_ERROR_STOP("tajo.cli.error.stop", false),
 
     // Timezone & Date ----------------------------------------------------------
-    $TIMEZONE("tajo.timezone", TimeZone.getDefault().getID()),
+    $TIMEZONE("tajo.timezone", TajoConstants.DEFAULT_SYSTEM_TIMEZONE),
     $DATE_ORDER("tajo.date.order", "YMD"),
 
     // FILE FORMAT

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
index 442db71..cfff369 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
@@ -100,7 +100,7 @@ public abstract class Datum implements Comparable<Datum>, GsonObject {
   }
 
   public byte[] asTextBytes() {
-    return toString().getBytes();
+    return asChars().getBytes();
   }
 
   public boolean isNumeric() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
index 17cfc7a..11ba791 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
@@ -27,7 +27,9 @@ import org.apache.tajo.util.datetime.DateTimeFormat;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
+import java.util.TimeZone;
 
 public class DatumFactory {
 
@@ -282,6 +284,12 @@ public class DatumFactory {
     return new TimeDatum(DateTimeUtil.toJulianTime(timeStr));
   }
 
+  public static TimeDatum createTime(String timeStr, TimeZone tz) {
+    TimeMeta tm = DateTimeUtil.decodeDateTime(timeStr);
+    DateTimeUtil.toUTCTimezone(tm, tz);
+    return new TimeDatum(DateTimeUtil.toTime(tm));
+  }
+
   public static TimestampDatum createTimestmpDatumWithJavaMillis(long millis) {
     return new TimestampDatum(DateTimeUtil.javaTimeToJulianTime(millis));
   }
@@ -294,6 +302,12 @@ public class DatumFactory {
     return new TimestampDatum(DateTimeUtil.toJulianTimestamp(datetimeStr));
   }
 
+  public static TimestampDatum createTimestamp(String datetimeStr, TimeZone tz) {
+    TimeMeta tm = DateTimeUtil.decodeDateTime(datetimeStr);
+    DateTimeUtil.toUTCTimezone(tm, tz);
+    return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
+  }
+
   public static IntervalDatum createInterval(String intervalStr) {
     return new IntervalDatum(intervalStr);
   }
@@ -318,13 +332,17 @@ public class DatumFactory {
     }
   }
 
-  public static TimeDatum createTime(Datum datum) {
+  public static TimeDatum createTime(Datum datum, @Nullable TimeZone tz) {
     switch (datum.type()) {
     case INT8:
       return new TimeDatum(datum.asInt8());
+    case CHAR:
+    case VARCHAR:
     case TEXT:
       TimeMeta tm = DateTimeFormat.parseDateTime(datum.asChars(), "HH24:MI:SS.MS");
-      DateTimeUtil.toUTCTimezone(tm);
+      if (tz != null) {
+        DateTimeUtil.toUTCTimezone(tm, tz);
+      }
       return new TimeDatum(DateTimeUtil.toTime(tm));
     case TIME:
       return (TimeDatum) datum;
@@ -333,10 +351,12 @@ public class DatumFactory {
     }
   }
 
-  public static TimestampDatum createTimestamp(Datum datum) {
+  public static TimestampDatum createTimestamp(Datum datum, @Nullable TimeZone tz) {
     switch (datum.type()) {
+      case CHAR:
+      case VARCHAR:
       case TEXT:
-        return parseTimestamp(datum.asChars());
+        return parseTimestamp(datum.asChars(), tz);
       case TIMESTAMP:
         return (TimestampDatum) datum;
       default:
@@ -349,8 +369,8 @@ public class DatumFactory {
     return new TimestampDatum(julianTimestamp);
   }
 
-  public static TimestampDatum parseTimestamp(String str) {
-    return new TimestampDatum(DateTimeUtil.toJulianTimestampWithTZ(str));
+  public static TimestampDatum parseTimestamp(String str, @Nullable TimeZone tz) {
+    return new TimestampDatum(DateTimeUtil.toJulianTimestampWithTZ(str, tz));
   }
 
   public static BlobDatum createBlob(byte[] val) {
@@ -381,7 +401,7 @@ public class DatumFactory {
     return new Inet4Datum(val);
   }
 
-  public static Datum cast(Datum operandDatum, DataType target) {
+  public static Datum cast(Datum operandDatum, DataType target, @Nullable TimeZone tz) {
     switch (target.getType()) {
     case BOOLEAN:
       return DatumFactory.createBool(operandDatum.asBool());
@@ -398,21 +418,24 @@ public class DatumFactory {
       return DatumFactory.createFloat4(operandDatum.asFloat4());
     case FLOAT8:
       return DatumFactory.createFloat8(operandDatum.asFloat8());
+    case VARCHAR:
     case TEXT:
       switch (operandDatum.type()) {
         case TIMESTAMP: {
           TimestampDatum timestampDatum = (TimestampDatum)operandDatum;
-          TimeMeta tm = timestampDatum.toTimeMeta();
-          DateTimeUtil.toUserTimezone(tm);
-          TimestampDatum convertedTimestampDatum = new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
-          return DatumFactory.createText(convertedTimestampDatum.asTextBytes());
+          if (tz != null) {
+            return DatumFactory.createText(timestampDatum.asChars(tz, false));
+          } else {
+            return DatumFactory.createText(timestampDatum.asChars());
+          }
         }
         case TIME: {
           TimeDatum timeDatum = (TimeDatum)operandDatum;
-          TimeMeta tm = timeDatum.toTimeMeta();
-          DateTimeUtil.toUserTimezone(tm);
-          TimeDatum convertedTimeDatum = new TimeDatum(DateTimeUtil.toTime(tm));
-          return DatumFactory.createText(convertedTimeDatum.asTextBytes());
+          if (tz != null) {
+            return DatumFactory.createText(timeDatum.asChars(tz, false));
+          } else {
+            return DatumFactory.createText(timeDatum.asChars());
+          }
         }
         default:
           return DatumFactory.createText(operandDatum.asTextBytes());
@@ -420,9 +443,9 @@ public class DatumFactory {
     case DATE:
       return DatumFactory.createDate(operandDatum);
     case TIME:
-      return DatumFactory.createTime(operandDatum);
+      return DatumFactory.createTime(operandDatum, tz);
     case TIMESTAMP:
-      return DatumFactory.createTimestamp(operandDatum);
+      return DatumFactory.createTimestamp(operandDatum, tz);
     case BLOB:
       return DatumFactory.createBlob(operandDatum.asByteArray());
     case INET4:

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java b/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java
new file mode 100644
index 0000000..38441d3
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java
@@ -0,0 +1,40 @@
+/*
+ * 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.json;
+
+import com.google.gson.*;
+
+import java.lang.reflect.Type;
+import java.util.TimeZone;
+
+public class TimeZoneGsonSerdeAdapter implements GsonSerDerAdapter<TimeZone> {
+
+	@Override
+	public JsonElement serialize(TimeZone object, Type arg1, JsonSerializationContext arg2) {
+		return new JsonPrimitive(object.getID());
+	}
+
+  @Override
+  public TimeZone deserialize(JsonElement arg0, Type arg1, JsonDeserializationContext context) throws JsonParseException {
+    return TimeZone.getTimeZone(arg0.getAsJsonPrimitive().getAsString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index 459c9c9..d2c6c1c 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -18,11 +18,20 @@
 
 package org.apache.tajo.storage;
 
+import org.apache.tajo.TajoConstants;
+
 public class StorageConstants {
-  // table properties
+
+  // Common table properties -------------------------------------------------
+
+  // time zone
+  public static final String TIMEZONE = "timezone";
+
+  // compression
   public static final String COMPRESSION_CODEC = "compression.codec";
   public static final String COMPRESSION_TYPE = "compression.type";
 
+  // Text file properties -------------------------------------------------
   @Deprecated
   public static final String CSVFILE_DELIMITER = "csvfile.delimiter";
   @Deprecated
@@ -47,12 +56,14 @@ public class StorageConstants {
   public static final String TEXT_ERROR_TOLERANCE_MAXNUM = "text.error-tolerance.max-num";
   public static final String DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM = "0";
 
+  // Sequence file properties -------------------------------------------------
   @Deprecated
   public static final String SEQUENCEFILE_DELIMITER = "sequencefile.delimiter";
   @Deprecated
   public static final String SEQUENCEFILE_NULL = "sequencefile.null";
   public static final String SEQUENCEFILE_SERDE = "sequencefile.serde";
 
+  // RC file properties -------------------------------------------------
   @Deprecated
   public static final String RCFILE_NULL = "rcfile.null";
   public static final String RCFILE_SERDE = "rcfile.serde";
@@ -61,17 +72,22 @@ public class StorageConstants {
   public static final String DEFAULT_BINARY_SERDE = "org.apache.tajo.storage.BinarySerializerDeserializer";
   public static final String DEFAULT_TEXT_SERDE = "org.apache.tajo.storage.TextSerializerDeserializer";
 
+
+  // Parquet file properties -------------------------------------------------
   public static final String PARQUET_DEFAULT_BLOCK_SIZE;
   public static final String PARQUET_DEFAULT_PAGE_SIZE;
   public static final String PARQUET_DEFAULT_COMPRESSION_CODEC_NAME;
   public static final String PARQUET_DEFAULT_IS_DICTIONARY_ENABLED;
   public static final String PARQUET_DEFAULT_IS_VALIDATION_ENABLED;
 
+  public static final int DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024;
+  public static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024;
+
+
+  // Avro file properties -------------------------------------------------
   public static final String AVRO_SCHEMA_LITERAL = "avro.schema.literal";
   public static final String AVRO_SCHEMA_URL = "avro.schema.url";
 
-  public static final int DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024;
-  public static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024;
   static {
     PARQUET_DEFAULT_BLOCK_SIZE = Integer.toString(DEFAULT_BLOCK_SIZE);
     PARQUET_DEFAULT_PAGE_SIZE = Integer.toString(DEFAULT_PAGE_SIZE);

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
index a9dc4e8..7acb544 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
@@ -25,6 +25,7 @@ import org.apache.tajo.util.datetime.DateTimeConstants.DateStyle;
 import org.apache.tajo.util.datetime.DateTimeConstants.DateToken;
 import org.apache.tajo.util.datetime.DateTimeConstants.TokenField;
 
+import javax.annotation.Nullable;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -34,7 +35,6 @@ import java.util.concurrent.atomic.AtomicLong;
  * This Class is originated from j2date in datetime.c of PostgreSQL.
  */
 public class DateTimeUtil {
-
   private static int MAX_FRACTION_LENGTH = 6;
 
   /** maximum possible number of fields in a date * string */
@@ -663,16 +663,20 @@ public class DateTimeUtil {
 
 
   /**
-   * Parse datetime string to julian time.
-   * The result is the local time basis.
+   * Parse datetime string to UTC-based julian time.
+   * The result julian time is adjusted by local timezone.
+   *
    * @param timestampStr
-   * @return
+   * @param tz Local timezone. If it is NULL, UTC will be used by default.
+   * @return UTC-based julian time
    */
-  public static long toJulianTimestampWithTZ(String timestampStr) {
+  public static long toJulianTimestampWithTZ(String timestampStr, @Nullable TimeZone tz) {
     long timestamp = DateTimeUtil.toJulianTimestamp(timestampStr);
     TimeMeta tm = new TimeMeta();
     DateTimeUtil.toJulianTimeMeta(timestamp, tm);
-    DateTimeUtil.toUTCTimezone(tm);
+    if (tz != null) {
+      DateTimeUtil.toUTCTimezone(tm, tz);
+    }
     return DateTimeUtil.toJulianTimestamp(tm);
   }
 
@@ -2060,16 +2064,11 @@ public class DateTimeUtil {
     return date2j(year, mon, mday) - isoweek2j(date2isoyear(year, mon, mday), 1) + 1;
   }
 
-  public static void toUserTimezone(TimeMeta tm) {
-    toUserTimezone(tm, TajoConf.getCurrentTimeZone());
-  }
-
   public static void toUserTimezone(TimeMeta tm, TimeZone timeZone) {
     tm.plusMillis(timeZone.getRawOffset());
   }
 
-  public static void toUTCTimezone(TimeMeta tm) {
-    TimeZone timeZone = TajoConf.getCurrentTimeZone();
+  public static void toUTCTimezone(TimeMeta tm, TimeZone timeZone) {
     tm.plusMillis(0 - timeZone.getRawOffset());
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
index 5886083..277eefc 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.datum;
 
 import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.exception.InvalidCastException;
 import org.apache.tajo.json.CommonGsonHelper;
 import org.apache.tajo.util.datetime.DateTimeUtil;
@@ -29,8 +28,8 @@ import org.junit.Test;
 import java.util.Calendar;
 import java.util.TimeZone;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.*;
-import static org.hamcrest.CoreMatchers.*;
 
 public class TestTimestampDatum {
   private static long javatime;
@@ -40,7 +39,7 @@ public class TestTimestampDatum {
   @BeforeClass
   public static void setUp() {
     javatime = System.currentTimeMillis();
-    calendar = Calendar.getInstance(TajoConf.getCurrentTimeZone());
+    calendar = Calendar.getInstance(TimeZone.getTimeZone("PST"));
     calendar.setTimeInMillis(javatime);
     unixtime = (int) (javatime / 1000);
   }
@@ -151,7 +150,7 @@ public class TestTimestampDatum {
     assertEquals(datum2, datum);
 
     for (int i = 0; i < 100; i++) {
-      TimeZone timeZone = TimeZone.getTimeZone("UTC");
+      TimeZone timeZone = TimeZone.getTimeZone("GMT");
       Calendar cal = Calendar.getInstance(timeZone);
       long jTime = System.currentTimeMillis();
       int uTime = (int)(jTime / 1000);

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
index 5961076..f337c36 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
@@ -52,9 +52,6 @@ public class AvgDouble extends AggFunction {
     return new AvgContext();
   }
 
-  public void init() {
-  }
-
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     AvgContext avgCtx = (AvgContext) ctx;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
index f43a41f..ccfbd84 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
@@ -18,17 +18,24 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.DateDatum;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.plan.expr.FunctionEval;
+import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 @Description(
     functionName = "current_date",
     description = "Get current date. Result is DATE type.",
@@ -37,19 +44,26 @@ import org.apache.tajo.util.datetime.TimeMeta;
     paramTypes = {@ParamTypes(paramTypes = {})}
 )
 public class CurrentDate extends GeneralFunction {
-  DateDatum datum;
+  @Expose private TimeZone timezone;
+  private DateDatum datum;
 
   public CurrentDate() {
     super(NoArgs);
   }
 
   @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] types) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
+
+  @Override
   public Datum eval(Tuple params) {
     if (datum == null) {
       long julianTimestamp = DateTimeUtil.javaTimeToJulianTime(System.currentTimeMillis());
       TimeMeta tm = new TimeMeta();
       DateTimeUtil.toJulianTimeMeta(julianTimestamp, tm);
-      DateTimeUtil.toUserTimezone(tm);
+      DateTimeUtil.toUserTimezone(tm, timezone);
       datum = DatumFactory.createDate(tm.years, tm.monthOfYear, tm.dayOfMonth);
     }
     return datum;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
index 66a54bf..ae09197 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
@@ -18,20 +18,27 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.datum.TimeDatum;
-import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.plan.expr.FunctionEval;
+import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.datetime.DateTimeConstants;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.FLOAT8;
 import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 
@@ -44,6 +51,9 @@ import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
     paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TIME})}
 )
 public class DatePartFromTime extends GeneralFunction {
+  @Expose  private TimeZone timezone;
+  private DatePartExtractorFromTime extractor = null;
+
   public DatePartFromTime() {
     super(new Column[] {
         new Column("target", FLOAT8),
@@ -51,7 +61,11 @@ public class DatePartFromTime extends GeneralFunction {
     });
   }
 
-  private DatePartExtractorFromTime extractor = null;
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType [] types) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
 
   @Override
   public Datum eval(Tuple params) {
@@ -93,7 +107,7 @@ public class DatePartFromTime extends GeneralFunction {
     }
 
     TimeMeta tm = time.toTimeMeta();
-    DateTimeUtil.toUserTimezone(tm);
+    DateTimeUtil.toUserTimezone(tm, timezone);
     return extractor.extract(tm);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
index 843c593..cd195fb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
@@ -18,9 +18,14 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.*;
+import org.apache.tajo.plan.expr.FunctionEval;
 import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
@@ -29,6 +34,8 @@ import org.apache.tajo.util.datetime.DateTimeConstants;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.*;
 
 @Description(
@@ -40,6 +47,9 @@ import static org.apache.tajo.common.TajoDataTypes.Type.*;
     paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TIMESTAMP})}
 )
 public class DatePartFromTimestamp extends GeneralFunction {
+  @Expose private TimeZone timezone;
+  private DatePartExtractorFromTimestamp extractor = null;
+
   public DatePartFromTimestamp() {
     super(new Column[] {
         new Column("target", FLOAT8),
@@ -47,7 +57,11 @@ public class DatePartFromTimestamp extends GeneralFunction {
     });
   }
 
-  private DatePartExtractorFromTimestamp extractor = null;
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType [] types) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
 
   @Override
   public Datum eval(Tuple params) {
@@ -115,7 +129,7 @@ public class DatePartFromTimestamp extends GeneralFunction {
     }
 
     TimeMeta tm = timestamp.toTimeMeta();
-    DateTimeUtil.toUserTimezone(tm);
+    DateTimeUtil.toUserTimezone(tm, timezone);
 
     return extractor.extract(tm);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
index 02b52e3..9e76677 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
@@ -18,6 +18,10 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -33,6 +37,8 @@ import org.apache.tajo.util.datetime.DateTimeFormat;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 import static org.apache.tajo.common.TajoDataTypes.Type.TIMESTAMP;
 
@@ -45,6 +51,8 @@ import static org.apache.tajo.common.TajoDataTypes.Type.TIMESTAMP;
   paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TIMESTAMP, TajoDataTypes.Type.TEXT})}
 )
 public class ToCharTimestamp extends GeneralFunction {
+  @Expose private TimeZone timezone;
+
   public ToCharTimestamp() {
     super(new Column[] {
         new Column("timestamp", TIMESTAMP),
@@ -53,7 +61,9 @@ public class ToCharTimestamp extends GeneralFunction {
   }
 
   @Override
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
   }
 
   @Override
@@ -63,11 +73,11 @@ public class ToCharTimestamp extends GeneralFunction {
     }
 
     TimestampDatum valueDatum = (TimestampDatum) params.get(0);
-    TimeMeta tm = valueDatum.toTimeMeta();
-    DateTimeUtil.toUserTimezone(tm);
-
     Datum pattern = params.get(1);
 
+    TimeMeta tm = valueDatum.toTimeMeta();
+    DateTimeUtil.toUserTimezone(tm, timezone);
+
     return DatumFactory.createText(DateTimeFormat.to_char(tm, pattern.asChars()));
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
index e50bacc..e9beefb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
@@ -18,9 +18,13 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.*;
+import org.apache.tajo.plan.expr.FunctionEval;
 import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
@@ -29,6 +33,8 @@ import org.apache.tajo.util.datetime.DateTimeFormat;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 
 @Description(
@@ -41,10 +47,17 @@ import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
     paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
 )
 public class ToTimestampText extends GeneralFunction {
+  private TimeZone timezone;
+
   public ToTimestampText() {
     super(new Column[]{new Column("DateTimeText", TEXT), new Column("Pattern", TEXT)});
   }
 
+  public void init(OverridableConf queryContext, FunctionEval.ParamType [] paramTypes) {
+    String timezoneId = queryContext.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
+
   @Override
   public Datum eval(Tuple params) {
     if(params.isNull(0) || params.isNull(1)) {
@@ -55,7 +68,7 @@ public class ToTimestampText extends GeneralFunction {
     TextDatum patternDatum = (TextDatum) params.get(1);
 
     TimeMeta tm = DateTimeFormat.parseDateTime(dateTimeTextDatum.asChars(), patternDatum.asChars());
-    DateTimeUtil.toUTCTimezone(tm);
+    DateTimeUtil.toUTCTimezone(tm, timezone);
 
     return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
index 3214a5e..9864b0d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -57,7 +58,8 @@ public class BTrim extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 2) {
       hasTrimCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
index f345293..5d95c38 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -57,7 +58,8 @@ public class LTrim extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 2) {
       hasTrimCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
index 58c0951..da11f19 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -57,7 +58,8 @@ public class Lpad extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 3) {
       hasFillCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
index 8e295fa..03b9c25 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -56,7 +57,8 @@ public class RTrim extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 2) {
       hasTrimCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
index da06f73..6b888ec 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.BooleanDatum;
@@ -70,7 +71,8 @@ public class RegexpReplace extends GeneralFunction {
     });
   }
 
-  public void init(ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, ParamType[] paramTypes) {
     if (paramTypes[0] == ParamType.NULL || paramTypes[1] == ParamType.NULL || paramTypes[2] == ParamType.NULL) {
       isAlwaysNull = true;
     } else if (paramTypes[1] == ParamType.CONSTANT) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
index 9d42848..1509118 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -58,7 +59,8 @@ public class Rpad extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 3) {
       hasFillCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
index 57f2536..5a03bfd 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
@@ -38,6 +38,7 @@ import org.apache.tajo.util.TUtil;
 
 import java.lang.reflect.Type;
 import java.util.Map;
+import java.util.TimeZone;
 
 public class CoreGsonHelper {
   private static Gson gson;
@@ -58,6 +59,7 @@ public class CoreGsonHelper {
     adapters.put(AggFunction.class, new FunctionAdapter());
     adapters.put(Datum.class, new DatumAdapter());
     adapters.put(DataType.class, new DataTypeAdapter());
+    adapters.put(TimeZone.class, new TimeZoneGsonSerdeAdapter());
 
     return adapters;
 	}

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
index d8f7f08..47ead40 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
@@ -74,12 +74,12 @@ public class QueryContext extends OverridableConf {
   }
 
   public QueryContext(TajoConf conf, Session session) {
-    super(conf);
+    super(conf, ConfigKey.ConfigType.QUERY, ConfigKey.ConfigType.SESSION);
     putAll(session.getAllVariables());
   }
 
   public QueryContext(TajoConf conf, KeyValueSetProto proto) {
-    super(conf, proto, ConfigKey.ConfigType.QUERY);
+    super(conf, proto, ConfigKey.ConfigType.QUERY, ConfigKey.ConfigType.SESSION);
   }
 
   //-----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
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 4e4964e..efcc691 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -532,6 +532,14 @@ public class QueryTestCaseBase {
    *   replaced by the first and second elements of <code>args</code> respectively</li>. It uses zero-based index.
    * </ul>
    *
+   * Example ddl
+   * <pre>
+   *   CREATE EXTERNAL TABLE ${0} (
+   *     t_timestamp  TIMESTAMP,
+   *     t_date    DATE
+   *   ) USING CSV LOCATION ${table.path}
+   * </pre>
+   *
    * @param ddlFileName A file name, containing a data definition statement.
    * @param dataFileName A file name, containing data rows, which columns have to be separated by vertical bar '|'.
    *                     This file name is used for replacing some format string indicating an external table location.

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
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 603da0c..e5dfb7a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -59,6 +59,7 @@ import java.net.URL;
 import java.sql.ResultSet;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.TimeZone;
 import java.util.UUID;
 
 public class TajoTestingCluster {
@@ -117,6 +118,8 @@ public class TajoTestingCluster {
   }
 
   void initPropertiesAndConfigs() {
+    TimeZone.setDefault(TimeZone.getTimeZone(TajoConstants.DEFAULT_SYSTEM_TIMEZONE));
+
     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()));

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index df9ef65..8cd87a8 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -58,6 +58,7 @@ import org.junit.BeforeClass;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.TimeZone;
 
 import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
 import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
@@ -73,8 +74,8 @@ public class ExprTestBase {
   private static LogicalOptimizer optimizer;
   private static LogicalPlanVerifier annotatedPlanVerifier;
 
-  public static String getUserTimeZoneDisplay() {
-    return DateTimeUtil.getTimeZoneDisplayTime(TajoConf.getCurrentTimeZone());
+  public static String getUserTimeZoneDisplay(TimeZone tz) {
+    return DateTimeUtil.getTimeZoneDisplayTime(tz);
   }
 
   public ExprTestBase() {
@@ -164,7 +165,7 @@ public class ExprTestBase {
       assertJsonSerDer(t.getEvalTree());
     }
     for (Target t : targets) {
-      assertEvalTreeProtoSerDer(t.getEvalTree());
+      assertEvalTreeProtoSerDer(context, t.getEvalTree());
     }
     return targets;
   }
@@ -173,8 +174,19 @@ public class ExprTestBase {
     testEval(null, null, null, query, expected);
   }
 
-  public void testSimpleEval(String query, String [] expected, boolean condition) throws IOException {
-    testEval(null, null, null, null, query, expected, ',', condition);
+  public void testSimpleEval(OverridableConf context, String query, String [] expected) throws IOException {
+    testEval(context, null, null, null, query, expected);
+  }
+
+  public void testSimpleEval(String query, String [] expected, boolean successOrFail)
+      throws IOException {
+
+    testEval(null, null, null, null, query, expected, ',', successOrFail);
+  }
+
+  public void testSimpleEval(OverridableConf context, String query, String [] expected, boolean successOrFail)
+      throws IOException {
+    testEval(context, null, null, null, query, expected, ',', successOrFail);
   }
 
   public void testEval(Schema schema, String tableName, String csvTuple, String query, String [] expected)
@@ -183,10 +195,10 @@ public class ExprTestBase {
         expected, ',', true);
   }
 
-  public void testEval(OverridableConf overideConf, Schema schema, String tableName, String csvTuple, String query,
+  public void testEval(OverridableConf context, Schema schema, String tableName, String csvTuple, String query,
                        String [] expected)
       throws IOException {
-    testEval(overideConf, schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple,
+    testEval(context, schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple,
         query, expected, ',', true);
   }
 
@@ -196,16 +208,19 @@ public class ExprTestBase {
         query, expected, delimiter, condition);
   }
 
-  public void testEval(OverridableConf overideConf, Schema schema, String tableName, String csvTuple, String query,
+  public void testEval(OverridableConf context, Schema schema, String tableName, String csvTuple, String query,
                        String [] expected, char delimiter, boolean condition) throws IOException {
-    QueryContext context;
-    if (overideConf == null) {
-      context = LocalTajoTestingUtility.createDummyContext(conf);
+    QueryContext queryContext;
+    if (context == null) {
+      queryContext = LocalTajoTestingUtility.createDummyContext(conf);
     } else {
-      context = LocalTajoTestingUtility.createDummyContext(conf);
-      context.putAll(overideConf);
+      queryContext = LocalTajoTestingUtility.createDummyContext(conf);
+      queryContext.putAll(context);
     }
 
+    String timezoneId = queryContext.get(SessionVars.TZ);
+    TimeZone timeZone = TimeZone.getTimeZone(timezoneId);
+
     LazyTuple lazyTuple;
     VTuple vtuple  = null;
     String qualifiedTableName =
@@ -230,8 +245,8 @@ public class ExprTestBase {
         boolean nullDatum;
         Datum datum = lazyTuple.get(i);
         nullDatum = (datum instanceof TextDatum || datum instanceof CharDatum);
-        nullDatum = nullDatum && datum.asChars().equals("") ||
-            datum.asChars().equals(context.get(SessionVars.NULL_CHAR));
+        nullDatum = nullDatum &&
+            datum.asChars().equals("") || datum.asChars().equals(queryContext.get(SessionVars.NULL_CHAR));
         nullDatum |= datum.isNull();
 
         if (nullDatum) {
@@ -249,10 +264,10 @@ public class ExprTestBase {
     TajoClassLoader classLoader = new TajoClassLoader();
 
     try {
-      targets = getRawTargets(context, query, condition);
+      targets = getRawTargets(queryContext, query, condition);
 
       EvalCodeGenerator codegen = null;
-      if (context.getBool(SessionVars.CODEGEN)) {
+      if (queryContext.getBool(SessionVars.CODEGEN)) {
         codegen = new EvalCodeGenerator(classLoader);
       }
 
@@ -260,7 +275,7 @@ public class ExprTestBase {
       for (int i = 0; i < targets.length; i++) {
         EvalNode eval = targets[i].getEvalTree();
 
-        if (context.getBool(SessionVars.CODEGEN)) {
+        if (queryContext.getBool(SessionVars.CODEGEN)) {
           eval = codegen.compile(inputSchema, eval);
         }
 
@@ -277,9 +292,9 @@ public class ExprTestBase {
         Datum datum = outTuple.get(i);
         String outTupleAsChars;
         if (datum.type() == Type.TIMESTAMP) {
-          outTupleAsChars = ((TimestampDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true);
+          outTupleAsChars = ((TimestampDatum) datum).asChars(timeZone, false);
         } else if (datum.type() == Type.TIME) {
-          outTupleAsChars = ((TimeDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true);
+          outTupleAsChars = ((TimeDatum) datum).asChars(timeZone, false);
         } else {
           outTupleAsChars = datum.asChars();
         }
@@ -302,8 +317,8 @@ public class ExprTestBase {
     }
   }
 
-  public static void assertEvalTreeProtoSerDer(EvalNode evalNode) {
+  public static void assertEvalTreeProtoSerDer(OverridableConf context, EvalNode evalNode) {
     PlanProto.EvalTree converted = EvalTreeProtoSerializer.serialize(evalNode);
-    assertEquals(evalNode, EvalTreeProtoDeserializer.deserialize(converted));
+    assertEquals(evalNode, EvalTreeProtoDeserializer.deserialize(context, converted));
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
index c054fd1..cde370d 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
@@ -32,28 +32,28 @@ public class TestIntervalType extends ExprTestBase {
     // http://www.postgresql.org/docs/8.2/static/functions-datetime.html
     testSimpleEval("select date '2001-09-28' + 7", new String[]{"2001-10-05"});
     testSimpleEval("select date '2001-09-28' + interval '1 hour'",
-        new String[]{"2001-09-28 01:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 01:00:00"});
 
     testSimpleEval("select date '2001-09-28' + time '03:00'",
-        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 03:00:00"});
     testSimpleEval("select time '03:00' + date '2001-09-28'",
-        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 03:00:00"});
     testSimpleEval("select interval '1 day' + interval '1 hour'", new String[]{"1 day 01:00:00"});
 
     testSimpleEval("select timestamp '2001-09-28 01:00' + interval '23 hours'",
-        new String[]{"2001-09-29 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-29 00:00:00"});
 
-    testSimpleEval("select time '01:00' + interval '3 hours'", new String[]{"04:00:00" + getUserTimeZoneDisplay()});
+    testSimpleEval("select time '01:00' + interval '3 hours'", new String[]{"04:00:00"});
 
     testSimpleEval("select date '2001-10-01' - date '2001-09-28'", new String[]{"3"});
     testSimpleEval("select date '2001-10-01' - 7", new String[]{"2001-09-24"});
     testSimpleEval("select date '2001-09-28' - interval '1 hour'",
-        new String[]{"2001-09-27 23:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-27 23:00:00"});
 
     testSimpleEval("select time '05:00' - time '03:00'", new String[]{"02:00:00"});
-    testSimpleEval("select time '05:00' - interval '2 hours'", new String[]{"03:00:00" + getUserTimeZoneDisplay()});
+    testSimpleEval("select time '05:00' - interval '2 hours'", new String[]{"03:00:00"});
     testSimpleEval("select timestamp '2001-09-28 23:00' - interval '23 hours'",
-        new String[]{"2001-09-28 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 00:00:00"});
 
     testSimpleEval("select interval '1 day' - interval '1 hour'", new String[]{"23:00:00"});
 
@@ -67,32 +67,32 @@ public class TestIntervalType extends ExprTestBase {
   @Test
   public void testCaseByCase() throws Exception {
     testSimpleEval("select date '2001-08-28' + interval '10 day 1 hour'",
-        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-07 01:00:00"});
     testSimpleEval("select interval '10 day 01:00:00' + date '2001-08-28'",
-        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-07 01:00:00"});
     testSimpleEval("select time '10:20:30' + interval '1 day 01:00:00'",
-        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"11:20:30"});
     testSimpleEval("select interval '1 day 01:00:00' + time '10:20:30'",
-        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"11:20:30"});
     testSimpleEval("select time '10:20:30' - interval '1 day 01:00:00'",
-        new String[]{"09:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"09:20:30"});
 
     testSimpleEval("select (interval '1 month 20 day' + interval '50 day')", new String[]{"1 month 70 days"});
     testSimpleEval("select date '2013-01-01' + interval '1 month 70 day'",
-        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-04-12 00:00:00"});
     testSimpleEval("select date '2013-01-01' + (interval '1 month 20 day' + interval '50 day')",
-        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-04-12 00:00:00"});
     testSimpleEval("select interval '1 month 70 day' + date '2013-01-01'",
-        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-04-12 00:00:00"});
     testSimpleEval("select date '2013-01-01' - interval '1 month 70 day'",
-        new String[]{"2012-09-22 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2012-09-22 00:00:00"});
 
     testSimpleEval("select timestamp '2001-09-28 23:00' - interval '1 month 2 day 10:20:30'",
-        new String[]{"2001-08-26 12:39:30" + getUserTimeZoneDisplay()});
+        new String[]{"2001-08-26 12:39:30"});
     testSimpleEval("select timestamp '2001-09-28 23:00' + interval '1 month 2 day 10:20:30'",
-        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"2001-10-31 09:20:30"});
     testSimpleEval("select interval '1 month 2 day 10:20:30' + timestamp '2001-09-28 23:00'",
-        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"2001-10-31 09:20:30"});
 
 
     testSimpleEval("select interval '5 month' / 3", new String[]{"1 month 20 days"});
@@ -104,13 +104,13 @@ public class TestIntervalType extends ExprTestBase {
     testSimpleEval("select interval '3 year 5 month 1 hour' / 1.5", new String[]{"2 years 3 months 10 days 00:40:00"});
 
     testSimpleEval("select date '2001-09-28' - time '03:00'",
-        new String[]{"2001-09-27 21:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-27 21:00:00"});
 
     testSimpleEval("select date '2014-03-20' + interval '1 day'",
-        new String[]{"2014-03-21 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-03-21 00:00:00"});
 
     testSimpleEval("select date '2014-03-20' - interval '1 day'",
-        new String[]{"2014-03-19 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-03-19 00:00:00"});
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
index 60f7dcd..fc74339 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
@@ -26,14 +26,10 @@ public class TestSQLDateTimeTypes extends ExprTestBase {
 
   @Test
   public void testTimestamp() throws IOException {
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37';",
-        new String[]{"1970-01-17 10:09:37" + getUserTimeZoneDisplay()});
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.5';",
-        new String[]{"1970-01-17 10:09:37.5" + getUserTimeZoneDisplay()});
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.01';",
-        new String[]{"1970-01-17 10:09:37.01" + getUserTimeZoneDisplay()});
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.003';",
-        new String[]{"1970-01-17 10:09:37.003" + getUserTimeZoneDisplay()});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37';", new String[]{"1970-01-17 10:09:37"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.5';", new String[]{"1970-01-17 10:09:37.5"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.01';", new String[]{"1970-01-17 10:09:37.01"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.003';",new String[]{"1970-01-17 10:09:37.003"});
   }
 
   @Test
@@ -44,14 +40,12 @@ public class TestSQLDateTimeTypes extends ExprTestBase {
 
   @Test
   public void testTimeLiteral() throws IOException {
-    testSimpleEval("select TIME '10:09:37';",
-        new String[]{"10:09:37" + getUserTimeZoneDisplay()});
+    testSimpleEval("select TIME '10:09:37';", new String[]{"10:09:37"});
   }
 
   @Test
   public void testDateLiteral() throws IOException {
-    testSimpleEval("select DATE '1970-01-17';",
-        new String[]{"1970-01-17"});
+    testSimpleEval("select DATE '1970-01-17';", new String[]{"1970-01-17"});
   }
 
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
----------------------------------------------------------------------
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 6c9892a..4a9e42c 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
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.engine.eval;
 
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.exception.NoSuchFunctionException;
@@ -25,6 +26,7 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.TimestampDatum;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.junit.Test;
 
@@ -853,41 +855,46 @@ public class TestSQLExpression extends ExprTestBase {
 
   @Test
   public void testCastWithNestedFunction() throws IOException {
-    int unixtime = 1389071574;//(int) (System.currentTimeMillis() / 1000);
+    QueryContext context = new QueryContext(getConf());
+    context.put(SessionVars.TZ, "GMT-6");
+    TimeZone tz = TimeZone.getTimeZone("GMT-6");
+
+    int unixtime = 1389071574; // (int) (System.currentTimeMillis() / 1000);
     TimestampDatum expected = DatumFactory.createTimestmpDatumWithUnixTime(unixtime);
-    testSimpleEval(String.format("select to_timestamp(CAST(split_part('%d.999', '.', 1) as INT8));", unixtime),
-        new String[] {expected.asChars(TajoConf.getCurrentTimeZone(), true)});
+    testSimpleEval(context, String.format("select to_timestamp(CAST(split_part('%d.999', '.', 1) as INT8));", unixtime),
+        new String[] {expected.asChars(tz, false)});
   }
 
   @Test
   public void testCastFromTable() throws IOException {
-    TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6"));
-    try {
-      Schema schema = new Schema();
-      schema.addColumn("col1", TEXT);
-      schema.addColumn("col2", TEXT);
-
-      testEval(schema, "table1", "123,234", "select cast(col1 as float) as b, cast(col2 as float) as a from table1",
-          new String[]{"123.0", "234.0"});
-      testEval(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(TajoConf.getCurrentTimeZone()));
-
-      testEval(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[]{timestamp.asChars(TajoConf.getCurrentTimeZone(), true), "234.0"}
-      );
-
-      testSimpleEval("select '1980-04-01 01:50:01'::timestamp;",
-          new String[]{timestamp.asChars(TajoConf.getCurrentTimeZone(), true)});
-      testSimpleEval("select '1980-04-01 01:50:01'::timestamp::text", new String[]{"1980-04-01 01:50:01"});
-
-      testSimpleEval("select (cast ('99999'::int8 as text))::int4 + 1", new String[]{"100000"});
-    } finally {
-      TajoConf.setCurrentTimeZone(originTimeZone);
-    }
+    QueryContext queryContext = new QueryContext(getConf());
+    queryContext.put(SessionVars.TZ, "GMT-6");
+    TimeZone tz = TimeZone.getTimeZone("GMT-6");
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+
+    testEval(queryContext, schema,
+        "table1",
+        "123,234",
+        "select cast(col1 as float) as b, cast(col2 as float) as a from table1",
+        new String[]{"123.0", "234.0"});
+    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, "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[]{timestamp.asChars(tz, false), "234.0"}
+    );
+
+    testSimpleEval("select '1980-04-01 01:50:01'::timestamp;", new String[]{timestamp.asChars(tz, false)});
+    testSimpleEval("select '1980-04-01 01:50:01'::timestamp::text", new String[]{"1980-04-01 01:50:01"});
+
+    testSimpleEval("select (cast ('99999'::int8 as text))::int4 + 1", new String[]{"100000"});
   }
 
   @Test


[09/10] tajo git commit: TAJO-1196: Unit test hangs occasionally and randomly. (jihoon)

Posted by ji...@apache.org.
TAJO-1196: Unit test hangs occasionally and randomly. (jihoon)


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

Branch: refs/heads/index_support
Commit: 8f68b4bafdbb216365459d5bd118534d0031e548
Parents: 88e5c9e
Author: Jihoon Son <ji...@apache.org>
Authored: Tue Dec 9 10:54:07 2014 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Tue Dec 9 10:54:07 2014 +0900

----------------------------------------------------------------------
 CHANGES                                                      | 2 ++
 .../java/org/apache/tajo/master/querymaster/QueryMaster.java | 8 ++++----
 2 files changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/8f68b4ba/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 581c997..59a8f68 100644
--- a/CHANGES
+++ b/CHANGES
@@ -87,6 +87,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1196: Unit test hangs occasionally and randomly. (jihoon)
+
     TAJO-1234: Rearrange timezone in date/time types. (hyunsik)
 
     TAJO-1231: Implicit table properties in session are not stored in 

http://git-wip-us.apache.org/repos/asf/tajo/blob/8f68b4ba/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
index b3b4dbb..f2d8b3a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -480,6 +480,10 @@ public class QueryMaster extends CompositeService implements EventHandler {
       QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext,
           event.getQueryId(), event.getSession(), event.getQueryContext(), event.getJsonExpr(), event.getLogicalPlanJson());
 
+      synchronized(queryMasterTasks) {
+        queryMasterTasks.put(event.getQueryId(), queryMasterTask);
+      }
+
       queryMasterTask.init(systemConf);
       if (!queryMasterTask.isInitError()) {
         queryMasterTask.start();
@@ -487,10 +491,6 @@ public class QueryMaster extends CompositeService implements EventHandler {
 
       queryContext = event.getQueryContext();
 
-      synchronized(queryMasterTasks) {
-        queryMasterTasks.put(event.getQueryId(), queryMasterTask);
-      }
-
       if (queryMasterTask.isInitError()) {
         queryMasterContext.stopQuery(queryMasterTask.getQueryId());
         return;


[05/10] tajo git commit: TAJO-1229: rename tajo-yarn-pullserver to tajo-pullserver.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
deleted file mode 100644
index 1c63c8a..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
+++ /dev/null
@@ -1,654 +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.pullserver;
-
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.ReadaheadPool;
-import org.apache.hadoop.metrics2.MetricsSystem;
-import org.apache.hadoop.metrics2.annotation.Metric;
-import org.apache.hadoop.metrics2.annotation.Metrics;
-import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.apache.hadoop.metrics2.lib.MutableCounterInt;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
-import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
-import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
-import org.apache.hadoop.yarn.server.api.AuxiliaryService;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.pullserver.retriever.FileChunk;
-import org.apache.tajo.storage.RowStoreUtil;
-import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.TupleComparator;
-import org.apache.tajo.storage.index.bst.BSTIndex;
-import org.apache.tajo.util.TajoIdUtils;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.*;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
-import org.jboss.netty.handler.codec.frame.TooLongFrameException;
-import org.jboss.netty.handler.codec.http.*;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.handler.stream.ChunkedWriteHandler;
-import org.jboss.netty.util.CharsetUtil;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
-import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-
-public class PullServerAuxService extends AuxiliaryService {
-
-  private static final Log LOG = LogFactory.getLog(PullServerAuxService.class);
-  
-  public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache";
-  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
-
-  public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes";
-  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
-
-  private int port;
-  private ChannelFactory selector;
-  private final ChannelGroup accepted = new DefaultChannelGroup();
-  private HttpPipelineFactory pipelineFact;
-  private int sslFileBufferSize;
-
-  private ApplicationId appId;
-  private QueryId queryId;
-  private FileSystem localFS;
-
-  /**
-   * Should the shuffle use posix_fadvise calls to manage the OS cache during
-   * sendfile
-   */
-  private boolean manageOsCache;
-  private int readaheadLength;
-  private ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
-   
-
-  public static final String PULLSERVER_SERVICEID = "tajo.pullserver";
-
-  private static final Map<String,String> userRsrc =
-    new ConcurrentHashMap<String,String>();
-  private static String userName;
-
-  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
-    "tajo.pullserver.ssl.file.buffer.size";
-
-  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
-
-  @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo")
-  static class ShuffleMetrics implements ChannelFutureListener {
-    @Metric({"OutputBytes","PullServer output in bytes"})
-    MutableCounterLong shuffleOutputBytes;
-    @Metric({"Failed","# of failed shuffle outputs"})
-    MutableCounterInt shuffleOutputsFailed;
-    @Metric({"Succeeded","# of succeeded shuffle outputs"})
-    MutableCounterInt shuffleOutputsOK;
-    @Metric({"Connections","# of current shuffle connections"})
-    MutableGaugeInt shuffleConnections;
-
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      if (future.isSuccess()) {
-        shuffleOutputsOK.incr();
-      } else {
-        shuffleOutputsFailed.incr();
-      }
-      shuffleConnections.decr();
-    }
-  }
-
-  final ShuffleMetrics metrics;
-
-  PullServerAuxService(MetricsSystem ms) {
-    super("httpshuffle");
-    metrics = ms.register(new ShuffleMetrics());
-  }
-
-  @SuppressWarnings("UnusedDeclaration")
-  public PullServerAuxService() {
-    this(DefaultMetricsSystem.instance());
-  }
-
-  /**
-   * Serialize the shuffle port into a ByteBuffer for use later on.
-   * @param port the port to be sent to the ApplciationMaster
-   * @return the serialized form of the port.
-   */
-  public static ByteBuffer serializeMetaData(int port) throws IOException {
-    //TODO these bytes should be versioned
-    DataOutputBuffer port_dob = new DataOutputBuffer();
-    port_dob.writeInt(port);
-    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
-  }
-
-  /**
-   * A helper function to deserialize the metadata returned by PullServerAuxService.
-   * @param meta the metadata returned by the PullServerAuxService
-   * @return the port the PullServer Handler is listening on to serve shuffle data.
-   */
-  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
-    //TODO this should be returning a class not just an int
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(meta);
-    return in.readInt();
-  }
-
-  @Override
-  public void initializeApplication(ApplicationInitializationContext appInitContext) {
-    // TODO these bytes should be versioned
-    // TODO: Once SHuffle is out of NM, this can use MR APIs
-    this.appId = appInitContext.getApplicationId();
-    this.queryId = TajoIdUtils.parseQueryId(appId.toString());
-    this.userName = appInitContext.getUser();
-    userRsrc.put(this.appId.toString(), this.userName);
-  }
-
-  @Override
-  public void stopApplication(ApplicationTerminationContext appStopContext) {
-    userRsrc.remove(appStopContext.getApplicationId().toString());
-  }
-
-  @Override
-  public synchronized void init(Configuration conf) {
-    try {
-      manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
-          DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
-
-      readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
-          DEFAULT_SHUFFLE_READAHEAD_BYTES);
-
-      ThreadFactory bossFactory = new ThreadFactoryBuilder()
-          .setNameFormat("PullServerAuxService Netty Boss #%d")
-          .build();
-      ThreadFactory workerFactory = new ThreadFactoryBuilder()
-          .setNameFormat("PullServerAuxService Netty Worker #%d")
-          .build();
-
-      selector = new NioServerSocketChannelFactory(
-          Executors.newCachedThreadPool(bossFactory),
-          Executors.newCachedThreadPool(workerFactory));
-
-      localFS = new LocalFileSystem();
-      super.init(new Configuration(conf));
-    } catch (Throwable t) {
-      LOG.error(t);
-    }
-  }
-
-  // TODO change AbstractService to throw InterruptedException
-  @Override
-  public synchronized void start() {
-    Configuration conf = getConfig();
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-    try {
-      pipelineFact = new HttpPipelineFactory(conf);
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-    bootstrap.setPipelineFactory(pipelineFact);
-    port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
-        ConfVars.PULLSERVER_PORT.defaultIntVal);
-    Channel ch = bootstrap.bind(new InetSocketAddress(port));
-    accepted.add(ch);
-    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
-    conf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port));
-    pipelineFact.PullServer.setPort(port);
-    LOG.info(getName() + " listening on port " + port);
-    super.start();
-
-    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
-                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
-  }
-
-  public int getPort() {
-    return port;
-  }
-
-  @Override
-  public synchronized void stop() {
-    try {
-      accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
-      ServerBootstrap bootstrap = new ServerBootstrap(selector);
-      bootstrap.releaseExternalResources();
-      pipelineFact.destroy();
-
-      localFS.close();
-    } catch (Throwable t) {
-      LOG.error(t);
-    } finally {
-      super.stop();
-    }
-  }
-
-  @Override
-  public synchronized ByteBuffer getMetaData() {
-    try {
-      return serializeMetaData(port); 
-    } catch (IOException e) {
-      LOG.error("Error during getMeta", e);
-      // TODO add API to AuxiliaryServices to report failures
-      return null;
-    }
-  }
-
-  class HttpPipelineFactory implements ChannelPipelineFactory {
-
-    final PullServer PullServer;
-    private SSLFactory sslFactory;
-
-    public HttpPipelineFactory(Configuration conf) throws Exception {
-      PullServer = new PullServer(conf);
-      if (conf.getBoolean(ConfVars.SHUFFLE_SSL_ENABLED_KEY.varname,
-          ConfVars.SHUFFLE_SSL_ENABLED_KEY.defaultBoolVal)) {
-        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
-        sslFactory.init();
-      }
-    }
-
-    public void destroy() {
-      if (sslFactory != null) {
-        sslFactory.destroy();
-      }
-    }
-
-    @Override
-    public ChannelPipeline getPipeline() throws Exception {
-      ChannelPipeline pipeline = Channels.pipeline();
-      if (sslFactory != null) {
-        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
-      }
-      pipeline.addLast("decoder", new HttpRequestDecoder());
-      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
-      pipeline.addLast("encoder", new HttpResponseEncoder());
-      pipeline.addLast("chunking", new ChunkedWriteHandler());
-      pipeline.addLast("shuffle", PullServer);
-      return pipeline;
-      // TODO factor security manager into pipeline
-      // TODO factor out encode/decode to permit binary shuffle
-      // TODO factor out decode of index to permit alt. models
-    }
-  }
-
-  class PullServer extends SimpleChannelUpstreamHandler {
-    private final Configuration conf;
-    private final LocalDirAllocator lDirAlloc = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
-    private int port;
-
-    public PullServer(Configuration conf) {
-      this.conf = conf;
-      this.port = conf.getInt(ConfVars.PULLSERVER_PORT.varname, ConfVars.PULLSERVER_PORT.defaultIntVal);
-    }
-    
-    public void setPort(int port) {
-      this.port = port;
-    }
-
-    private List<String> splitMaps(List<String> mapq) {
-      if (null == mapq) {
-        return null;
-      }
-      final List<String> ret = new ArrayList<String>();
-      for (String s : mapq) {
-        Collections.addAll(ret, s.split(","));
-      }
-      return ret;
-    }
-
-    @Override
-    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
-        throws Exception {
-
-      HttpRequest request = (HttpRequest) e.getMessage();
-      if (request.getMethod() != GET) {
-        sendError(ctx, METHOD_NOT_ALLOWED);
-        return;
-      }
-
-      // Parsing the URL into key-values
-      final Map<String, List<String>> params =
-          new QueryStringDecoder(request.getUri()).getParameters();
-      final List<String> types = params.get("type");
-      final List<String> taskIdList = params.get("ta");
-      final List<String> subQueryIds = params.get("sid");
-      final List<String> partitionIds = params.get("p");
-
-      if (types == null || taskIdList == null || subQueryIds == null
-          || partitionIds == null) {
-        sendError(ctx, "Required type, taskIds, subquery Id, and partition id",
-            BAD_REQUEST);
-        return;
-      }
-
-      if (types.size() != 1 || subQueryIds.size() != 1) {
-        sendError(ctx, "Required type, taskIds, subquery Id, and partition id",
-            BAD_REQUEST);
-        return;
-      }
-
-      final List<FileChunk> chunks = Lists.newArrayList();
-
-      String repartitionType = types.get(0);
-      String sid = subQueryIds.get(0);
-      String partitionId = partitionIds.get(0);
-      List<String> taskIds = splitMaps(taskIdList);
-
-      // the working dir of tajo worker for each query
-      String queryBaseDir = queryId + "/output" + "/";
-
-      LOG.info("PullServer request param: repartitionType=" + repartitionType +
-          ", sid=" + sid + ", partitionId=" + partitionId + ", taskIds=" + taskIdList);
-
-      String taskLocalDir = conf.get(ConfVars.WORKER_TEMPORAL_DIR.varname);
-      if (taskLocalDir == null ||
-          taskLocalDir.equals("")) {
-        LOG.error("Tajo local directory should be specified.");
-      }
-      LOG.info("PullServer baseDir: " + taskLocalDir + "/" + queryBaseDir);
-
-      // if a subquery requires a range partitioning
-      if (repartitionType.equals("r")) {
-        String ta = taskIds.get(0);
-        Path path = localFS.makeQualified(
-            lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/"
-                + ta + "/output/", conf));
-
-        String startKey = params.get("start").get(0);
-        String endKey = params.get("end").get(0);
-        boolean last = params.get("final") != null;
-
-        FileChunk chunk;
-        try {
-          chunk = getFileCunks(path, startKey, endKey, last);
-        } catch (Throwable t) {
-          LOG.error("ERROR Request: " + request.getUri(), t);
-          sendError(ctx, "Cannot get file chunks to be sent", BAD_REQUEST);
-          return;
-        }
-        if (chunk != null) {
-          chunks.add(chunk);
-        }
-
-        // if a subquery requires a hash repartition  or a scattered hash repartition
-      } else if (repartitionType.equals("h") || repartitionType.equals("s")) {
-        for (String ta : taskIds) {
-          Path path = localFS.makeQualified(
-              lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" +
-                  ta + "/output/" + partitionId, conf));
-          File file = new File(path.toUri());
-          FileChunk chunk = new FileChunk(file, 0, file.length());
-          chunks.add(chunk);
-        }
-      } else {
-        LOG.error("Unknown repartition type: " + repartitionType);
-        return;
-      }
-
-      // Write the content.
-      Channel ch = e.getChannel();
-      if (chunks.size() == 0) {
-        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
-        ch.write(response);
-        if (!isKeepAlive(request)) {
-          ch.close();
-        }
-      }  else {
-        FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
-        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-        long totalSize = 0;
-        for (FileChunk chunk : file) {
-          totalSize += chunk.length();
-        }
-        setContentLength(response, totalSize);
-
-        // Write the initial line and the header.
-        ch.write(response);
-
-        ChannelFuture writeFuture = null;
-
-        for (FileChunk chunk : file) {
-          writeFuture = sendFile(ctx, ch, chunk);
-          if (writeFuture == null) {
-            sendError(ctx, NOT_FOUND);
-            return;
-          }
-        }
-
-        // Decide whether to close the connection or not.
-        if (!isKeepAlive(request)) {
-          // Close the connection when the whole content is written out.
-          writeFuture.addListener(ChannelFutureListener.CLOSE);
-        }
-      }
-    }
-
-    private ChannelFuture sendFile(ChannelHandlerContext ctx,
-                                   Channel ch,
-                                   FileChunk file) throws IOException {
-      RandomAccessFile spill;
-      try {
-        spill = new RandomAccessFile(file.getFile(), "r");
-      } catch (FileNotFoundException e) {
-        LOG.info(file.getFile() + " not found");
-        return null;
-      }
-      ChannelFuture writeFuture;
-      if (ch.getPipeline().get(SslHandler.class) == null) {
-        final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
-            file.startOffset(), file.length(), manageOsCache, readaheadLength,
-            readaheadPool, file.getFile().getAbsolutePath());
-        writeFuture = ch.write(partition);
-        writeFuture.addListener(new FileCloseListener(partition, null, 0, null));
-      } else {
-        // HTTPS cannot be done with zero copy.
-        final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
-            file.startOffset(), file.length(), sslFileBufferSize,
-            manageOsCache, readaheadLength, readaheadPool,
-            file.getFile().getAbsolutePath());
-        writeFuture = ch.write(chunk);
-      }
-      metrics.shuffleConnections.incr();
-      metrics.shuffleOutputBytes.incr(file.length()); // optimistic
-      return writeFuture;
-    }
-
-    private void sendError(ChannelHandlerContext ctx,
-        HttpResponseStatus status) {
-      sendError(ctx, "", status);
-    }
-
-    private void sendError(ChannelHandlerContext ctx, String message,
-        HttpResponseStatus status) {
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
-      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
-      response.setContent(
-        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
-
-      // Close the connection as soon as the error message is sent.
-      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
-        throws Exception {
-      Channel ch = e.getChannel();
-      Throwable cause = e.getCause();
-      if (cause instanceof TooLongFrameException) {
-        sendError(ctx, BAD_REQUEST);
-        return;
-      }
-
-      LOG.error("PullServer error: ", cause);
-      if (ch.isConnected()) {
-        LOG.error("PullServer error " + e);
-        sendError(ctx, INTERNAL_SERVER_ERROR);
-      }
-    }
-  }
-
-  public FileChunk getFileCunks(Path outDir,
-                                      String startKey,
-                                      String endKey,
-                                      boolean last) throws IOException {
-    BSTIndex index = new BSTIndex(new TajoConf());
-    BSTIndex.BSTIndexReader idxReader =
-        index.getIndexReader(new Path(outDir, "index"));
-    idxReader.open();
-    Schema keySchema = idxReader.getKeySchema();
-    TupleComparator comparator = idxReader.getComparator();
-
-    LOG.info("BSTIndex is loaded from disk (" + idxReader.getFirstKey() + ", "
-        + idxReader.getLastKey());
-
-    File data = new File(URI.create(outDir.toUri() + "/output"));
-    byte [] startBytes = Base64.decodeBase64(startKey);
-    byte [] endBytes = Base64.decodeBase64(endKey);
-
-    RowStoreDecoder decoder = RowStoreUtil.createDecoder(keySchema);
-    Tuple start;
-    Tuple end;
-    try {
-      start = decoder.toTuple(startBytes);
-    } catch (Throwable t) {
-      throw new IllegalArgumentException("StartKey: " + startKey
-          + ", decoded byte size: " + startBytes.length, t);
-    }
-
-    try {
-      end = decoder.toTuple(endBytes);
-    } catch (Throwable t) {
-      throw new IllegalArgumentException("EndKey: " + endKey
-          + ", decoded byte size: " + endBytes.length, t);
-    }
-
-
-    if(!comparator.isAscendingFirstKey()) {
-      Tuple tmpKey = start;
-      start = end;
-      end = tmpKey;
-    }
-
-    LOG.info("GET Request for " + data.getAbsolutePath() + " (start="+start+", end="+ end +
-        (last ? ", last=true" : "") + ")");
-
-    if (idxReader.getFirstKey() == null && idxReader.getLastKey() == null) { // if # of rows is zero
-      LOG.info("There is no contents");
-      return null;
-    }
-
-    if (comparator.compare(end, idxReader.getFirstKey()) < 0 ||
-        comparator.compare(idxReader.getLastKey(), start) < 0) {
-      LOG.info("Out of Scope (indexed data [" + idxReader.getFirstKey() + ", " + idxReader.getLastKey() +
-          "], but request start:" + start + ", end: " + end);
-      return null;
-    }
-
-    long startOffset;
-    long endOffset;
-    try {
-      startOffset = idxReader.find(start);
-    } catch (IOException ioe) {
-      LOG.error("State Dump (the requested range: "
-          + "[" + start + ", " + end +")" + ", idx min: "
-          + idxReader.getFirstKey() + ", idx max: "
-          + idxReader.getLastKey());
-      throw ioe;
-    }
-    try {
-      endOffset = idxReader.find(end);
-      if (endOffset == -1) {
-        endOffset = idxReader.find(end, true);
-      }
-    } catch (IOException ioe) {
-      LOG.error("State Dump (the requested range: "
-          + "[" + start + ", " + end +")" + ", idx min: "
-          + idxReader.getFirstKey() + ", idx max: "
-          + idxReader.getLastKey());
-      throw ioe;
-    }
-
-    // if startOffset == -1 then case 2-1 or case 3
-    if (startOffset == -1) { // this is a hack
-      // if case 2-1 or case 3
-      try {
-        startOffset = idxReader.find(start, true);
-      } catch (IOException ioe) {
-        LOG.error("State Dump (the requested range: "
-            + "[" + start + ", " + end +")" + ", idx min: "
-            + idxReader.getFirstKey() + ", idx max: "
-            + idxReader.getLastKey());
-        throw ioe;
-      }
-    }
-
-    if (startOffset == -1) {
-      throw new IllegalStateException("startOffset " + startOffset + " is negative \n" +
-          "State Dump (the requested range: "
-          + "[" + start + ", " + end +")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
-          + idxReader.getLastKey());
-    }
-
-    // if greater than indexed values
-    if (last || (endOffset == -1
-        && comparator.compare(idxReader.getLastKey(), end) < 0)) {
-      endOffset = data.length();
-    }
-
-    FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset);
-    LOG.info("Retrieve File Chunk: " + chunk);
-    return chunk;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java
deleted file mode 100644
index 564950f..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerUtil.java
+++ /dev/null
@@ -1,90 +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.pullserver;
-
-import org.apache.commons.lang.reflect.MethodUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.nativeio.NativeIO;
-
-import java.io.FileDescriptor;
-import java.lang.reflect.Method;
-
-public class PullServerUtil {
-  private static final Log LOG = LogFactory.getLog(PullServerUtil.class);
-
-  private static boolean nativeIOPossible = false;
-  private static Method posixFadviseIfPossible;
-
-  static {
-    if (NativeIO.isAvailable() && loadNativeIO()) {
-      nativeIOPossible = true;
-    } else {
-      LOG.warn("Unable to load hadoop nativeIO");
-    }
-  }
-
-  public static boolean isNativeIOPossible() {
-    return nativeIOPossible;
-  }
-
-  /**
-   * Call posix_fadvise on the given file descriptor. See the manpage
-   * for this syscall for more information. On systems where this
-   * call is not available, does nothing.
-   */
-  public static void posixFadviseIfPossible(String identifier, java.io.FileDescriptor fd,
-                                            long offset, long len, int flags) {
-    if (nativeIOPossible) {
-      try {
-        posixFadviseIfPossible.invoke(null, identifier, fd, offset, len, flags);
-      } catch (Throwable t) {
-        nativeIOPossible = false;
-        LOG.warn("Failed to manage OS cache for " + identifier, t);
-      }
-    }
-  }
-
-  /* load hadoop native method if possible */
-  private static boolean loadNativeIO() {
-    boolean loaded = true;
-    if (nativeIOPossible) return loaded;
-
-    Class[] parameters = {String.class, FileDescriptor.class, Long.TYPE, Long.TYPE, Integer.TYPE};
-    try {
-      Method getCacheManipulator = MethodUtils.getAccessibleMethod(NativeIO.POSIX.class, "getCacheManipulator", new Class[0]);
-      Class posixClass;
-      if (getCacheManipulator != null) {
-        Object posix = MethodUtils.invokeStaticMethod(NativeIO.POSIX.class, "getCacheManipulator", null);
-        posixClass = posix.getClass();
-      } else {
-        posixClass = NativeIO.POSIX.class;
-      }
-      posixFadviseIfPossible = MethodUtils.getAccessibleMethod(posixClass, "posixFadviseIfPossible", parameters);
-    } catch (Throwable e) {
-      loaded = false;
-      LOG.warn("Failed to access posixFadviseIfPossible :" + e.getMessage());
-    }
-
-    if (posixFadviseIfPossible == null) {
-      loaded = false;
-    }
-    return loaded;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java
deleted file mode 100644
index d030eed..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServer.java
+++ /dev/null
@@ -1,73 +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.pullserver;
-
-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.CompositeService;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.pullserver.PullServerAuxService.PullServer;
-import org.apache.tajo.util.StringUtils;
-
-public class TajoPullServer extends CompositeService {
-  private static final Log LOG = LogFactory.getLog(TajoPullServer.class);
-
-  private TajoPullServerService pullService;
-  private TajoConf systemConf;
-
-  public TajoPullServer() {
-    super(TajoPullServer.class.getName());
-  }
-
-  @Override
-  public void init(Configuration conf) {
-    this.systemConf = (TajoConf)conf;
-    pullService = new TajoPullServerService();
-    addService(pullService);
-
-    super.init(conf);
-  }
-
-  public void startPullServer(TajoConf systemConf) {
-    init(systemConf);
-    start();
-  }
-
-  public void start() {
-    super.start();
-
-  }
-
-  public static void main(String[] args) throws Exception {
-    StringUtils.startupShutdownMessage(PullServer.class, args, LOG);
-
-    if (!TajoPullServerService.isStandalone()) {
-      LOG.fatal("TAJO_PULLSERVER_STANDALONE env variable is not 'true'");
-      return;
-    }
-
-    TajoConf tajoConf = new TajoConf();
-    tajoConf.addResource(new Path(TajoConstants.SYSTEM_CONF_FILENAME));
-
-    (new TajoPullServer()).startPullServer(tajoConf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
deleted file mode 100644
index 5a4e69f..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
+++ /dev/null
@@ -1,808 +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.pullserver;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.ReadaheadPool;
-import org.apache.hadoop.metrics2.MetricsSystem;
-import org.apache.hadoop.metrics2.annotation.Metric;
-import org.apache.hadoop.metrics2.annotation.Metrics;
-import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.apache.hadoop.metrics2.lib.MutableCounterInt;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
-import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.pullserver.retriever.FileChunk;
-import org.apache.tajo.rpc.RpcChannelFactory;
-import org.apache.tajo.storage.HashShuffleAppenderManager;
-import org.apache.tajo.storage.RowStoreUtil;
-import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.TupleComparator;
-import org.apache.tajo.storage.index.bst.BSTIndex;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.*;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-import org.jboss.netty.handler.codec.http.*;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.handler.stream.ChunkedWriteHandler;
-import org.jboss.netty.util.CharsetUtil;
-
-import java.io.*;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
-import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-
-public class TajoPullServerService extends AbstractService {
-
-  private static final Log LOG = LogFactory.getLog(TajoPullServerService.class);
-
-  public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache";
-  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
-
-  public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes";
-  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
-
-  private int port;
-  private ChannelFactory selector;
-  private final ChannelGroup accepted = new DefaultChannelGroup();
-  private HttpPipelineFactory pipelineFact;
-  private int sslFileBufferSize;
-
-  private ApplicationId appId;
-  private FileSystem localFS;
-
-  /**
-   * Should the shuffle use posix_fadvise calls to manage the OS cache during
-   * sendfile
-   */
-  private boolean manageOsCache;
-  private int readaheadLength;
-  private ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
-
-
-  public static final String PULLSERVER_SERVICEID = "tajo.pullserver";
-
-  private static final Map<String,String> userRsrc =
-    new ConcurrentHashMap<String,String>();
-  private String userName;
-
-  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
-    "tajo.pullserver.ssl.file.buffer.size";
-
-  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
-
-  private static boolean STANDALONE = false;
-
-  static {
-    String standalone = System.getenv("TAJO_PULLSERVER_STANDALONE");
-    if (!StringUtils.isEmpty(standalone)) {
-      STANDALONE = standalone.equalsIgnoreCase("true");
-    }
-  }
-
-  @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo")
-  static class ShuffleMetrics implements ChannelFutureListener {
-    @Metric({"OutputBytes","PullServer output in bytes"})
-    MutableCounterLong shuffleOutputBytes;
-    @Metric({"Failed","# of failed shuffle outputs"})
-    MutableCounterInt shuffleOutputsFailed;
-    @Metric({"Succeeded","# of succeeded shuffle outputs"})
-    MutableCounterInt shuffleOutputsOK;
-    @Metric({"Connections","# of current shuffle connections"})
-    MutableGaugeInt shuffleConnections;
-
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      if (future.isSuccess()) {
-        shuffleOutputsOK.incr();
-      } else {
-        shuffleOutputsFailed.incr();
-      }
-      shuffleConnections.decr();
-    }
-  }
-
-  final ShuffleMetrics metrics;
-
-  TajoPullServerService(MetricsSystem ms) {
-    super("httpshuffle");
-    metrics = ms.register(new ShuffleMetrics());
-  }
-
-  @SuppressWarnings("UnusedDeclaration")
-  public TajoPullServerService() {
-    this(DefaultMetricsSystem.instance());
-  }
-
-  /**
-   * Serialize the shuffle port into a ByteBuffer for use later on.
-   * @param port the port to be sent to the ApplciationMaster
-   * @return the serialized form of the port.
-   */
-  public static ByteBuffer serializeMetaData(int port) throws IOException {
-    //TODO these bytes should be versioned
-    DataOutputBuffer port_dob = new DataOutputBuffer();
-    port_dob.writeInt(port);
-    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
-  }
-
-  /**
-   * A helper function to deserialize the metadata returned by PullServerAuxService.
-   * @param meta the metadata returned by the PullServerAuxService
-   * @return the port the PullServer Handler is listening on to serve shuffle data.
-   */
-  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
-    //TODO this should be returning a class not just an int
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(meta);
-    return in.readInt();
-  }
-
-  public void initApp(String user, ApplicationId appId, ByteBuffer secret) {
-    // TODO these bytes should be versioned
-    // TODO: Once SHuffle is out of NM, this can use MR APIs
-    this.appId = appId;
-    this.userName = user;
-    userRsrc.put(appId.toString(), user);
-  }
-
-  public void stopApp(ApplicationId appId) {
-    userRsrc.remove(appId.toString());
-  }
-
-  @Override
-  public void init(Configuration conf) {
-    try {
-      manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
-          DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
-
-      readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
-          DEFAULT_SHUFFLE_READAHEAD_BYTES);
-
-      int workerNum = conf.getInt("tajo.shuffle.rpc.server.worker-thread-num",
-          Runtime.getRuntime().availableProcessors() * 2);
-
-      selector = RpcChannelFactory.createServerChannelFactory("PullServerAuxService", workerNum);
-
-      localFS = new LocalFileSystem();
-
-      conf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname
-          , TajoConf.ConfVars.PULLSERVER_PORT.defaultIntVal);
-      super.init(conf);
-      LOG.info("Tajo PullServer initialized: readaheadLength=" + readaheadLength);
-    } catch (Throwable t) {
-      LOG.error(t);
-    }
-  }
-
-  // TODO change AbstractService to throw InterruptedException
-  @Override
-  public synchronized void serviceInit(Configuration conf) throws Exception {
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-
-    try {
-      pipelineFact = new HttpPipelineFactory(conf);
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-    bootstrap.setPipelineFactory(pipelineFact);
-
-    port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
-        ConfVars.PULLSERVER_PORT.defaultIntVal);
-    Channel ch = bootstrap.bind(new InetSocketAddress(port));
-
-    accepted.add(ch);
-    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
-    conf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port));
-    pipelineFact.PullServer.setPort(port);
-    LOG.info(getName() + " listening on port " + port);
-
-    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
-                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
-
-
-    if (STANDALONE) {
-      File pullServerPortFile = getPullServerPortFile();
-      if (pullServerPortFile.exists()) {
-        pullServerPortFile.delete();
-      }
-      pullServerPortFile.getParentFile().mkdirs();
-      LOG.info("Write PullServerPort to " + pullServerPortFile);
-      FileOutputStream out = null;
-      try {
-        out = new FileOutputStream(pullServerPortFile);
-        out.write(("" + port).getBytes());
-      } catch (Exception e) {
-        LOG.fatal("PullServer exists cause can't write PullServer port to " + pullServerPortFile +
-            ", " + e.getMessage(), e);
-        System.exit(-1);
-      } finally {
-        IOUtils.closeStream(out);
-      }
-    }
-    super.serviceInit(conf);
-    LOG.info("TajoPullServerService started: port=" + port);
-  }
-
-  public static boolean isStandalone() {
-    return STANDALONE;
-  }
-
-  private static File getPullServerPortFile() {
-    String pullServerPortInfoFile = System.getenv("TAJO_PID_DIR");
-    if (StringUtils.isEmpty(pullServerPortInfoFile)) {
-      pullServerPortInfoFile = "/tmp";
-    }
-    return new File(pullServerPortInfoFile + "/pullserver.port");
-  }
-
-  // TODO change to get port from master or tajoConf
-  public static int readPullServerPort() {
-    FileInputStream in = null;
-    try {
-      File pullServerPortFile = getPullServerPortFile();
-
-      if (!pullServerPortFile.exists() || pullServerPortFile.isDirectory()) {
-        return -1;
-      }
-      in = new FileInputStream(pullServerPortFile);
-      byte[] buf = new byte[1024];
-      int readBytes = in.read(buf);
-      return Integer.parseInt(new String(buf, 0, readBytes));
-    } catch (IOException e) {
-      LOG.fatal(e.getMessage(), e);
-      return -1;
-    } finally {
-      IOUtils.closeStream(in);
-    }
-  }
-
-  public int getPort() {
-    return port;
-  }
-
-  @Override
-  public synchronized void stop() {
-    try {
-      accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
-      ServerBootstrap bootstrap = new ServerBootstrap(selector);
-      bootstrap.releaseExternalResources();
-      pipelineFact.destroy();
-
-      localFS.close();
-    } catch (Throwable t) {
-      LOG.error(t);
-    } finally {
-      super.stop();
-    }
-  }
-
-  public synchronized ByteBuffer getMeta() {
-    try {
-      return serializeMetaData(port); 
-    } catch (IOException e) {
-      LOG.error("Error during getMeta", e);
-      // TODO add API to AuxiliaryServices to report failures
-      return null;
-    }
-  }
-
-  class HttpPipelineFactory implements ChannelPipelineFactory {
-
-    final PullServer PullServer;
-    private SSLFactory sslFactory;
-
-    public HttpPipelineFactory(Configuration conf) throws Exception {
-      PullServer = new PullServer(conf);
-      if (conf.getBoolean(ConfVars.SHUFFLE_SSL_ENABLED_KEY.varname,
-          ConfVars.SHUFFLE_SSL_ENABLED_KEY.defaultBoolVal)) {
-        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
-        sslFactory.init();
-      }
-    }
-
-    public void destroy() {
-      if (sslFactory != null) {
-        sslFactory.destroy();
-      }
-    }
-
-    @Override
-    public ChannelPipeline getPipeline() throws Exception {
-      ChannelPipeline pipeline = Channels.pipeline();
-      if (sslFactory != null) {
-        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
-      }
-
-      int maxChunkSize = getConfig().getInt(ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE.varname,
-          ConfVars.SHUFFLE_FETCHER_CHUNK_MAX_SIZE.defaultIntVal);
-      pipeline.addLast("codec", new HttpServerCodec(4096, 8192, maxChunkSize));
-      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
-      pipeline.addLast("chunking", new ChunkedWriteHandler());
-      pipeline.addLast("shuffle", PullServer);
-      return pipeline;
-      // TODO factor security manager into pipeline
-      // TODO factor out encode/decode to permit binary shuffle
-      // TODO factor out decode of index to permit alt. models
-    }
-  }
-
-
-  Map<String, ProcessingStatus> processingStatusMap = new ConcurrentHashMap<String, ProcessingStatus>();
-
-  public void completeFileChunk(FileRegion filePart,
-                                   String requestUri,
-                                   long startTime) {
-    ProcessingStatus status = processingStatusMap.get(requestUri);
-    if (status != null) {
-      status.decrementRemainFiles(filePart, startTime);
-    }
-  }
-
-  class ProcessingStatus {
-    String requestUri;
-    int numFiles;
-    AtomicInteger remainFiles;
-    long startTime;
-    long makeFileListTime;
-    long minTime = Long.MAX_VALUE;
-    long maxTime;
-    int numSlowFile;
-
-    public ProcessingStatus(String requestUri) {
-      this.requestUri = requestUri;
-      this.startTime = System.currentTimeMillis();
-    }
-
-    public void setNumFiles(int numFiles) {
-      this.numFiles = numFiles;
-      this.remainFiles = new AtomicInteger(numFiles);
-    }
-    public void decrementRemainFiles(FileRegion filePart, long fileStartTime) {
-      synchronized(remainFiles) {
-        long fileSendTime = System.currentTimeMillis() - fileStartTime;
-        if (fileSendTime > 20 * 1000) {
-          LOG.info("PullServer send too long time: filePos=" + filePart.getPosition() + ", fileLen=" + filePart.getCount());
-          numSlowFile++;
-        }
-        if (fileSendTime > maxTime) {
-          maxTime = fileSendTime;
-        }
-        if (fileSendTime < minTime) {
-          minTime = fileSendTime;
-        }
-        int remain = remainFiles.decrementAndGet();
-        if (remain <= 0) {
-          processingStatusMap.remove(requestUri);
-          LOG.info("PullServer processing status: totalTime=" + (System.currentTimeMillis() - startTime) + " ms, " +
-              "makeFileListTime=" + makeFileListTime + " ms, minTime=" + minTime + " ms, maxTime=" + maxTime + " ms, " +
-              "numFiles=" + numFiles + ", numSlowFile=" + numSlowFile);
-        }
-      }
-    }
-  }
-
-  class PullServer extends SimpleChannelUpstreamHandler {
-
-    private final Configuration conf;
-//    private final IndexCache indexCache;
-    private final LocalDirAllocator lDirAlloc =
-      new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
-    private int port;
-
-    public PullServer(Configuration conf) throws IOException {
-      this.conf = conf;
-//      indexCache = new IndexCache(new JobConf(conf));
-      this.port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
-          ConfVars.PULLSERVER_PORT.defaultIntVal);
-
-      // init local temporal dir
-      lDirAlloc.getAllLocalPathsToRead(".", conf);
-    }
-    
-    public void setPort(int port) {
-      this.port = port;
-    }
-
-    private List<String> splitMaps(List<String> mapq) {
-      if (null == mapq) {
-        return null;
-      }
-      final List<String> ret = new ArrayList<String>();
-      for (String s : mapq) {
-        Collections.addAll(ret, s.split(","));
-      }
-      return ret;
-    }
-
-    @Override
-    public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt)
-        throws Exception {
-
-      accepted.add(evt.getChannel());
-      LOG.info(String.format("Current number of shuffle connections (%d)", accepted.size()));
-      super.channelOpen(ctx, evt);
-
-    }
-
-    @Override
-    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
-        throws Exception {
-
-      HttpRequest request = (HttpRequest) e.getMessage();
-      if (request.getMethod() != GET) {
-        sendError(ctx, METHOD_NOT_ALLOWED);
-        return;
-      }
-
-      ProcessingStatus processingStatus = new ProcessingStatus(request.getUri().toString());
-      processingStatusMap.put(request.getUri().toString(), processingStatus);
-      // Parsing the URL into key-values
-      final Map<String, List<String>> params =
-          new QueryStringDecoder(request.getUri()).getParameters();
-      final List<String> types = params.get("type");
-      final List<String> qids = params.get("qid");
-      final List<String> taskIdList = params.get("ta");
-      final List<String> subQueryIds = params.get("sid");
-      final List<String> partIds = params.get("p");
-      final List<String> offsetList = params.get("offset");
-      final List<String> lengthList = params.get("length");
-
-      if (types == null || subQueryIds == null || qids == null || partIds == null) {
-        sendError(ctx, "Required queryId, type, subquery Id, and part id",
-            BAD_REQUEST);
-        return;
-      }
-
-      if (qids.size() != 1 && types.size() != 1 || subQueryIds.size() != 1) {
-        sendError(ctx, "Required qids, type, taskIds, subquery Id, and part id",
-            BAD_REQUEST);
-        return;
-      }
-
-      String partId = partIds.get(0);
-      String queryId = qids.get(0);
-      String shuffleType = types.get(0);
-      String sid = subQueryIds.get(0);
-
-      long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L;
-      long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L;
-
-      if (!shuffleType.equals("h") && !shuffleType.equals("s") && taskIdList == null) {
-        sendError(ctx, "Required taskIds", BAD_REQUEST);
-      }
-
-      List<String> taskIds = splitMaps(taskIdList);
-
-      String queryBaseDir = queryId.toString() + "/output";
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("PullServer request param: shuffleType=" + shuffleType +
-            ", sid=" + sid + ", partId=" + partId + ", taskIds=" + taskIdList);
-
-        // the working dir of tajo worker for each query
-        LOG.debug("PullServer baseDir: " + conf.get(ConfVars.WORKER_TEMPORAL_DIR.varname) + "/" + queryBaseDir);
-      }
-
-      final List<FileChunk> chunks = Lists.newArrayList();
-
-      // if a subquery requires a range shuffle
-      if (shuffleType.equals("r")) {
-        String ta = taskIds.get(0);
-        if(!lDirAlloc.ifExists(queryBaseDir + "/" + sid + "/" + ta + "/output/", conf)){
-          LOG.warn(e);
-          sendError(ctx, NO_CONTENT);
-          return;
-        }
-        Path path = localFS.makeQualified(
-            lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" + ta + "/output/", conf));
-        String startKey = params.get("start").get(0);
-        String endKey = params.get("end").get(0);
-        boolean last = params.get("final") != null;
-
-        FileChunk chunk;
-        try {
-          chunk = getFileCunks(path, startKey, endKey, last);
-        } catch (Throwable t) {
-          LOG.error("ERROR Request: " + request.getUri(), t);
-          sendError(ctx, "Cannot get file chunks to be sent", BAD_REQUEST);
-          return;
-        }
-        if (chunk != null) {
-          chunks.add(chunk);
-        }
-
-        // if a subquery requires a hash 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 + "/" + sid + "/hash-shuffle/" + partParentId + "/" + partId;
-        if (!lDirAlloc.ifExists(partPath, conf)) {
-          LOG.warn("Partition shuffle file not exists: " + partPath);
-          sendError(ctx, NO_CONTENT);
-          return;
-        }
-
-        Path path = localFS.makeQualified(lDirAlloc.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()) {
-          String errorMessage = "Start pos[" + startPos + "] great than file length [" + file.length() + "]";
-          LOG.error(errorMessage);
-          sendError(ctx, errorMessage, BAD_REQUEST);
-          return;
-        }
-        LOG.info("RequestURL: " + request.getUri() + ", fileLen=" + file.length());
-        FileChunk chunk = new FileChunk(file, startPos, readLen);
-        chunks.add(chunk);
-      } else {
-        LOG.error("Unknown shuffle type: " + shuffleType);
-        sendError(ctx, "Unknown shuffle type:" + shuffleType, BAD_REQUEST);
-        return;
-      }
-
-      processingStatus.setNumFiles(chunks.size());
-      processingStatus.makeFileListTime = System.currentTimeMillis() - processingStatus.startTime;
-      // Write the content.
-      Channel ch = e.getChannel();
-      if (chunks.size() == 0) {
-        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
-        ch.write(response);
-        if (!isKeepAlive(request)) {
-          ch.close();
-        }
-      }  else {
-        FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
-        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-        long totalSize = 0;
-        for (FileChunk chunk : file) {
-          totalSize += chunk.length();
-        }
-        setContentLength(response, totalSize);
-
-        // Write the initial line and the header.
-        ch.write(response);
-
-        ChannelFuture writeFuture = null;
-
-        for (FileChunk chunk : file) {
-          writeFuture = sendFile(ctx, ch, chunk, request.getUri().toString());
-          if (writeFuture == null) {
-            sendError(ctx, NOT_FOUND);
-            return;
-          }
-        }
-
-        // Decide whether to close the connection or not.
-        if (!isKeepAlive(request)) {
-          // Close the connection when the whole content is written out.
-          writeFuture.addListener(ChannelFutureListener.CLOSE);
-        }
-      }
-    }
-
-    private ChannelFuture sendFile(ChannelHandlerContext ctx,
-                                   Channel ch,
-                                   FileChunk file,
-                                   String requestUri) throws IOException {
-      long startTime = System.currentTimeMillis();
-      RandomAccessFile spill = null;
-      ChannelFuture writeFuture;
-      try {
-        spill = new RandomAccessFile(file.getFile(), "r");
-        if (ch.getPipeline().get(SslHandler.class) == null) {
-          final FadvisedFileRegion filePart = new FadvisedFileRegion(spill,
-              file.startOffset(), file.length(), manageOsCache, readaheadLength,
-              readaheadPool, file.getFile().getAbsolutePath());
-          writeFuture = ch.write(filePart);
-          writeFuture.addListener(new FileCloseListener(filePart, requestUri, startTime, TajoPullServerService.this));
-        } else {
-          // HTTPS cannot be done with zero copy.
-          final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
-              file.startOffset(), file.length(), sslFileBufferSize,
-              manageOsCache, readaheadLength, readaheadPool,
-              file.getFile().getAbsolutePath());
-          writeFuture = ch.write(chunk);
-        }
-      } catch (FileNotFoundException e) {
-        LOG.info(file.getFile() + " not found");
-        return null;
-      } catch (Throwable e) {
-        if (spill != null) {
-          //should close a opening file
-          spill.close();
-        }
-        return null;
-      }
-      metrics.shuffleConnections.incr();
-      metrics.shuffleOutputBytes.incr(file.length()); // optimistic
-      return writeFuture;
-    }
-
-    private void sendError(ChannelHandlerContext ctx,
-        HttpResponseStatus status) {
-      sendError(ctx, "", status);
-    }
-
-    private void sendError(ChannelHandlerContext ctx, String message,
-        HttpResponseStatus status) {
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
-      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
-      response.setContent(
-        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
-
-      // Close the connection as soon as the error message is sent.
-      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
-        throws Exception {
-      LOG.error(e.getCause().getMessage(), e.getCause());
-      //if channel.close() is not called, never closed files in this request
-      if (ctx.getChannel().isConnected()){
-        ctx.getChannel().close();
-      }
-    }
-  }
-
-  public static FileChunk getFileCunks(Path outDir,
-                                      String startKey,
-                                      String endKey,
-                                      boolean last) throws IOException {
-    BSTIndex index = new BSTIndex(new TajoConf());
-    BSTIndex.BSTIndexReader idxReader =
-        index.getIndexReader(new Path(outDir, "index"));
-    idxReader.open();
-    Schema keySchema = idxReader.getKeySchema();
-    TupleComparator comparator = idxReader.getComparator();
-
-    LOG.info("BSTIndex is loaded from disk (" + idxReader.getFirstKey() + ", "
-        + idxReader.getLastKey());
-
-    File data = new File(URI.create(outDir.toUri() + "/output"));
-    byte [] startBytes = Base64.decodeBase64(startKey);
-    byte [] endBytes = Base64.decodeBase64(endKey);
-
-    RowStoreDecoder decoder = RowStoreUtil.createDecoder(keySchema);
-    Tuple start;
-    Tuple end;
-    try {
-      start = decoder.toTuple(startBytes);
-    } catch (Throwable t) {
-      throw new IllegalArgumentException("StartKey: " + startKey
-          + ", decoded byte size: " + startBytes.length, t);
-    }
-
-    try {
-      end = decoder.toTuple(endBytes);
-    } catch (Throwable t) {
-      throw new IllegalArgumentException("EndKey: " + endKey
-          + ", decoded byte size: " + endBytes.length, t);
-    }
-
-    LOG.info("GET Request for " + data.getAbsolutePath() + " (start="+start+", end="+ end +
-        (last ? ", last=true" : "") + ")");
-
-    if (idxReader.getFirstKey() == null && idxReader.getLastKey() == null) { // if # of rows is zero
-      LOG.info("There is no contents");
-      return null;
-    }
-
-    if (comparator.compare(end, idxReader.getFirstKey()) < 0 ||
-        comparator.compare(idxReader.getLastKey(), start) < 0) {
-      LOG.warn("Out of Scope (indexed data [" + idxReader.getFirstKey() + ", " + idxReader.getLastKey() +
-          "], but request start:" + start + ", end: " + end);
-      return null;
-    }
-
-    long startOffset;
-    long endOffset;
-    try {
-      startOffset = idxReader.find(start);
-    } catch (IOException ioe) {
-      LOG.error("State Dump (the requested range: "
-          + "[" + start + ", " + end +")" + ", idx min: "
-          + idxReader.getFirstKey() + ", idx max: "
-          + idxReader.getLastKey());
-      throw ioe;
-    }
-    try {
-      endOffset = idxReader.find(end);
-      if (endOffset == -1) {
-        endOffset = idxReader.find(end, true);
-      }
-    } catch (IOException ioe) {
-      LOG.error("State Dump (the requested range: "
-          + "[" + start + ", " + end +")" + ", idx min: "
-          + idxReader.getFirstKey() + ", idx max: "
-          + idxReader.getLastKey());
-      throw ioe;
-    }
-
-    // if startOffset == -1 then case 2-1 or case 3
-    if (startOffset == -1) { // this is a hack
-      // if case 2-1 or case 3
-      try {
-        startOffset = idxReader.find(start, true);
-      } catch (IOException ioe) {
-        LOG.error("State Dump (the requested range: "
-            + "[" + start + ", " + end +")" + ", idx min: "
-            + idxReader.getFirstKey() + ", idx max: "
-            + idxReader.getLastKey());
-        throw ioe;
-      }
-    }
-
-    if (startOffset == -1) {
-      throw new IllegalStateException("startOffset " + startOffset + " is negative \n" +
-          "State Dump (the requested range: "
-          + "[" + start + ", " + end +")" + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
-          + idxReader.getLastKey());
-    }
-
-    // if greater than indexed values
-    if (last || (endOffset == -1
-        && comparator.compare(idxReader.getLastKey(), end) < 0)) {
-      endOffset = data.length();
-    }
-
-    idxReader.close();
-
-    FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset);
-    LOG.info("Retrieve File Chunk: " + chunk);
-    return chunk;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
deleted file mode 100644
index 67e7423..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
+++ /dev/null
@@ -1,126 +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.pullserver.retriever;
-
-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.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.pullserver.FileAccessForbiddenException;
-import org.apache.tajo.util.TajoIdUtils;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-public class AdvancedDataRetriever implements DataRetriever {
-  private final Log LOG = LogFactory.getLog(AdvancedDataRetriever.class);
-  private final Map<String, RetrieverHandler> handlerMap = Maps.newConcurrentMap();
-
-  public AdvancedDataRetriever() {
-  }
-  
-  public void register(String taskAttemptId, RetrieverHandler handler) {
-    synchronized (handlerMap) {
-      if (!handlerMap.containsKey(taskAttemptId)) {
-        handlerMap.put(taskAttemptId, handler);
-      }
-    } 
-  }
-  
-  public void unregister(String taskAttemptId) {
-    synchronized (handlerMap) {
-      if (handlerMap.containsKey(taskAttemptId)) {
-        handlerMap.remove(taskAttemptId);
-      }
-    }
-  }
-
-  @Override
-  public FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
-      throws IOException {
-
-    final Map<String, List<String>> params =
-      new QueryStringDecoder(request.getUri()).getParameters();
-
-    if (!params.containsKey("qid")) {
-      throw new FileNotFoundException("No such qid: " + params.containsKey("qid"));
-    }
-
-    if (params.containsKey("sid")) {
-      List<FileChunk> chunks = Lists.newArrayList();
-      List<String> queryUnidIds = splitMaps(params.get("qid"));
-      for (String eachQueryUnitId : queryUnidIds) {
-        String[] queryUnitIdSeqTokens = eachQueryUnitId.split("_");
-        ExecutionBlockId ebId = TajoIdUtils.createExecutionBlockId(params.get("sid").get(0));
-        QueryUnitId quid = new QueryUnitId(ebId, Integer.parseInt(queryUnitIdSeqTokens[0]));
-
-        QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid, Integer.parseInt(queryUnitIdSeqTokens[1]));
-
-        RetrieverHandler handler = handlerMap.get(attemptId.toString());
-        FileChunk chunk = handler.get(params);
-        chunks.add(chunk);
-      }
-      return chunks.toArray(new FileChunk[chunks.size()]);
-    } else {
-      RetrieverHandler handler = handlerMap.get(params.get("qid").get(0));
-      FileChunk chunk = handler.get(params);
-      if (chunk == null) {
-        if (params.containsKey("qid")) { // if there is no content corresponding to the query
-          return null;
-        } else { // if there is no
-          throw new FileNotFoundException("No such a file corresponding to " + params.get("qid"));
-        }
-      }
-
-      File file = chunk.getFile();
-      if (file.isHidden() || !file.exists()) {
-        throw new FileNotFoundException("No such file: " + file.getAbsolutePath());
-      }
-      if (!file.isFile()) {
-        throw new FileAccessForbiddenException(file.getAbsolutePath() + " is not file");
-      }
-
-      return new FileChunk[] {chunk};
-    }
-  }
-
-  private List<String> splitMaps(List<String> qids) {
-    if (null == qids) {
-      LOG.error("QueryUnitId is EMPTY");
-      return null;
-    }
-
-    final List<String> ret = new ArrayList<String>();
-    for (String qid : qids) {
-      Collections.addAll(ret, qid.split(","));
-    }
-    return ret;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.java
deleted file mode 100644
index 8f55f7b..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DataRetriever.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.pullserver.retriever;
-
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-
-import java.io.IOException;
-
-public interface DataRetriever {
-  FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
-      throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java
deleted file mode 100644
index dc63929..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/DirectoryRetriever.java
+++ /dev/null
@@ -1,56 +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.pullserver.retriever;
-
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.apache.tajo.pullserver.FileAccessForbiddenException;
-import org.apache.tajo.pullserver.HttpDataServerHandler;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-public class DirectoryRetriever implements DataRetriever {
-  public String baseDir;
-  
-  public DirectoryRetriever(String baseDir) {
-    this.baseDir = baseDir;
-  }
-
-  @Override
-  public FileChunk [] handle(ChannelHandlerContext ctx, HttpRequest request)
-      throws IOException {
-    final String path = HttpDataServerHandler.sanitizeUri(request.getUri());
-    if (path == null) {
-      throw new IllegalArgumentException("Wrong path: " +path);
-    }
-
-    File file = new File(baseDir, path);
-    if (file.isHidden() || !file.exists()) {
-      throw new FileNotFoundException("No such file: " + baseDir + "/" + path);
-    }
-    if (!file.isFile()) {
-      throw new FileAccessForbiddenException("No such file: "
-          + baseDir + "/" + path); 
-    }
-    
-    return new FileChunk[] {new FileChunk(file, 0, file.length())};
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java
deleted file mode 100644
index 67cff21..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/FileChunk.java
+++ /dev/null
@@ -1,81 +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.pullserver.retriever;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-
-public class FileChunk {
-  private final File file;
-  private final long startOffset;
-  private long length;
-
-  /**
-   * TRUE if this.file is created by getting data from a remote host (e.g., by HttpRequest). FALSE otherwise.
-   */
-  private boolean fromRemote;
-
-  /**
-   * ExecutionBlockId
-   */
-  private String ebId;
-
-  public FileChunk(File file, long startOffset, long length) throws FileNotFoundException {
-    this.file = file;
-    this.startOffset = startOffset;
-    this.length = length;
-  }
-
-  public File getFile() {
-    return this.file;
-  }
-
-  public long startOffset() {
-    return this.startOffset;
-  }
-
-  public long length() {
-    return this.length;
-  }
-
-  public void setLength(long newLength) {
-    this.length = newLength;
-  }
-
-  public boolean fromRemote() {
-    return this.fromRemote;
-  }
-
-  public void setFromRemote(boolean newVal) {
-    this.fromRemote = newVal;
-  }
-
-  public String getEbId() {
-    return this.ebId;
-  }
-
-  public void setEbId(String newVal) {
-    this.ebId = newVal;
-  }
-
-  public String toString() {
-    return " (start=" + startOffset() + ", length=" + length + ", fromRemote=" + fromRemote + ", ebId=" + ebId + ") "
-	+ file.getAbsolutePath();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/b5aa7804/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java b/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java
deleted file mode 100644
index 5567c0d..0000000
--- a/tajo-yarn-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/RetrieverHandler.java
+++ /dev/null
@@ -1,33 +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.pullserver.retriever;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-public interface RetrieverHandler {
-  /**
-   *
-   * @param kvs url-decoded key/value pairs
-   * @return a desired part of a file
-   * @throws java.io.IOException
-   */
-  public FileChunk get(Map<String, List<String>> kvs) throws IOException;
-}


[02/10] tajo git commit: TAJO-1213: Implement CatalogStore::updateTableStats. (jaehwa)

Posted by ji...@apache.org.
TAJO-1213: Implement CatalogStore::updateTableStats. (jaehwa)

Closes #285


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

Branch: refs/heads/index_support
Commit: 95cf4b9432a02fdbf9880b204c3db718e2bd2468
Parents: ab2efce
Author: JaeHwa Jung <bl...@apache.org>
Authored: Sun Dec 7 15:38:43 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Sun Dec 7 15:40:24 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  2 +
 .../tajo/catalog/AbstractCatalogClient.java     | 14 ++++
 .../src/main/proto/CatalogProtocol.proto        |  1 +
 .../org/apache/tajo/catalog/CatalogService.java |  5 ++
 .../src/main/proto/CatalogProtos.proto          |  5 ++
 .../tajo/catalog/store/HCatalogStore.java       |  6 ++
 .../org/apache/tajo/catalog/CatalogServer.java  | 22 +++++++
 .../tajo/catalog/store/AbstractDBStore.java     | 68 ++++++++++++++++++++
 .../apache/tajo/catalog/store/CatalogStore.java |  4 +-
 .../org/apache/tajo/catalog/store/MemStore.java | 17 +++++
 .../org/apache/tajo/master/GlobalEngine.java    |  8 ++-
 .../apache/tajo/master/querymaster/Query.java   |  9 ++-
 12 files changed, 156 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 8ecd795..a59e107 100644
--- a/CHANGES
+++ b/CHANGES
@@ -18,6 +18,8 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1213: Implement CatalogStore::updateTableStats. (jaehwa)
+
     TAJO-1165: Needs to show error messages on query_executor.jsp. 
     (Jihun Kang via jaehwa)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
index 1f1e808..dde6980 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
@@ -658,4 +658,18 @@ public abstract class AbstractCatalogClient implements CatalogService {
     }
   }
 
+  @Override
+  public boolean updateTableStats(final UpdateTableStatsProto updateTableStatsProto) {
+    try {
+      return new ServerCallable<Boolean>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+        public Boolean call(NettyClientBase client) throws ServiceException {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          return stub.updateTableStats(null, updateTableStatsProto).getValue();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return false;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
index c5cb528..adf0740 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
@@ -33,6 +33,7 @@ service CatalogProtocolService {
   rpc getTablespace(StringProto) returns (TablespaceProto);
   rpc alterTablespace(AlterTablespaceProto) returns (BoolProto);
   rpc alterTable(AlterTableDescProto) returns (BoolProto);
+  rpc updateTableStats(UpdateTableStatsProto) returns (BoolProto);
 
 
   rpc createDatabase(CreateDatabaseRequest) returns (BoolProto);

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
index 667ee88..b41b636 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
@@ -27,6 +27,8 @@ import java.util.Collection;
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto;
 import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import static org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
+import static org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
+
 
 public interface CatalogService {
 
@@ -181,4 +183,7 @@ public interface CatalogService {
   * @throws Throwable
   */
   boolean alterTable(AlterTableDesc desc);
+
+  boolean updateTableStats(UpdateTableStatsProto stats);
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index f29bc6c..22c08d8 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -280,6 +280,11 @@ message AlterColumnProto {
   required string newColumnName = 2;
 }
 
+message UpdateTableStatsProto {
+  required string tableName = 1;
+  required TableStatsProto stats = 2;
+}
+
 ////////////////////////////////////////////////
 // Function and UDF Section
 ////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
index fa1cfd6..ad0aee3 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
@@ -333,6 +333,12 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
+  public void updateTableStats(CatalogProtos.UpdateTableStatsProto statsProto) throws
+    CatalogException {
+    // TODO - not implemented yet
+  }
+
+  @Override
   public void alterTablespace(CatalogProtos.AlterTablespaceProto alterProto) throws CatalogException {
     throw new CatalogException("tablespace concept is not supported in HCatalogStore");
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
index 03ae920..57086e2 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
@@ -58,6 +58,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto.AlterTablespaceCommand;
 import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType.*;
 import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListProto;
+import static org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
 
 /**
  * This class provides the catalog service. The catalog service enables clients
@@ -366,6 +367,27 @@ public class CatalogServer extends AbstractService {
     }
 
     @Override
+    public BoolProto updateTableStats(RpcController controller, UpdateTableStatsProto proto) throws
+      ServiceException {
+      wlock.lock();
+      try {
+        String [] split = CatalogUtil.splitTableName(proto.getTableName());
+        if (!store.existTable(split[0], split[1])) {
+          throw new NoSuchTableException(proto.getTableName());
+        }
+        store.updateTableStats(proto);
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        return BOOL_FALSE;
+      } finally {
+        wlock.unlock();
+        LOG.info("Table " + proto.getTableName() + " is updated in the catalog ("
+          + bindAddressStr + ")");
+      }
+      return BOOL_TRUE;
+    }
+
+    @Override
     public BoolProto alterTable(RpcController controller, AlterTableDescProto proto) throws ServiceException {
       wlock.lock();
       try {

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
index 7c1baab..c7d55eb 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
@@ -823,6 +823,74 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
+  public void updateTableStats(final CatalogProtos.UpdateTableStatsProto statsProto) throws
+    CatalogException {
+    Connection conn = null;
+    PreparedStatement pstmt = null;
+    ResultSet res = null;
+
+    try {
+      conn = getConnection();
+      conn.setAutoCommit(false);
+
+      String[] splitted = CatalogUtil.splitTableName(statsProto.getTableName());
+      if (splitted.length == 1) {
+        throw new IllegalArgumentException("updateTableStats() requires a qualified table name, but it is \""
+          + statsProto.getTableName() + "\".");
+      }
+      String databaseName = splitted[0];
+      String tableName = splitted[1];
+
+      int dbid = getDatabaseId(databaseName);
+
+      String tidSql =
+        "SELECT TID from " + TB_TABLES + " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_NAME + "=?";
+      pstmt = conn.prepareStatement(tidSql);
+      pstmt.setInt(1, dbid);
+      pstmt.setString(2, tableName);
+      res = pstmt.executeQuery();
+
+      if (!res.next()) {
+        throw new CatalogException("ERROR: there is no TID matched to " + statsProto.getTableName());
+      }
+
+      int tableId = res.getInt("TID");
+      res.close();
+      pstmt.close();
+
+      if (statsProto.hasStats()) {
+
+        String statSql = "UPDATE " + TB_STATISTICS + " SET NUM_ROWS = ?, " +
+          "NUM_BYTES = ? WHERE TID = ?";
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(statSql);
+        }
+
+        pstmt = conn.prepareStatement(statSql);
+        pstmt.setInt(1, tableId);
+        pstmt.setLong(2, statsProto.getStats().getNumRows());
+        pstmt.setLong(3, statsProto.getStats().getNumBytes());
+        pstmt.executeUpdate();
+      }
+
+      // If there is no error, commit the changes.
+      conn.commit();
+    } catch (SQLException se) {
+      if (conn != null) {
+        try {
+          conn.rollback();
+        } catch (SQLException e) {
+          LOG.error(e);
+        }
+      }
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(pstmt, res);
+    }
+  }
+
+  @Override
   public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws CatalogException {
 
     String[] splitted = CatalogUtil.splitTableName(alterTableDescProto.getTableName());

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
index 5de9633..041fc52 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
@@ -68,7 +68,9 @@ public interface CatalogStore extends Closeable {
 
   void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws CatalogException;
 
-    /************************ PARTITION METHOD **************************/
+  void updateTableStats(CatalogProtos.UpdateTableStatsProto statsProto) throws CatalogException;
+
+  /************************ PARTITION METHOD **************************/
   void addPartitionMethod(PartitionMethodProto partitionMethodProto) throws CatalogException;
 
   PartitionMethodProto getPartitionMethod(String databaseName, String tableName)

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
index ca99160..9575c13 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
@@ -172,6 +172,23 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
+  public void updateTableStats(CatalogProtos.UpdateTableStatsProto request) throws CatalogException {
+    String [] splitted = CatalogUtil.splitTableName(request.getTableName());
+    if (splitted.length == 1) {
+      throw new IllegalArgumentException("createTable() requires a qualified table name, but it is \""
+        + request.getTableName() + "\".");
+    }
+    String databaseName = splitted[0];
+    String tableName = splitted[1];
+
+    final Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, databaseName);
+    final CatalogProtos.TableDescProto tableDescProto = database.get(tableName);
+    CatalogProtos.TableDescProto newTableDescProto = tableDescProto.toBuilder().setStats(request
+      .getStats().toBuilder()).build();
+    database.put(tableName, newTableDescProto);
+  }
+
+  @Override
   public boolean existTable(String dbName, String tbName) throws CatalogException {
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, dbName);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 9bf9a75..821d440 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -74,6 +74,7 @@ import java.util.List;
 
 import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto;
+import static org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
 import static org.apache.tajo.ipc.ClientProtos.SerializedResultSet;
 import static org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
 
@@ -398,8 +399,11 @@ public class GlobalEngine extends AbstractService {
       stats.setNumBytes(volume);
       stats.setNumRows(1);
 
-      catalog.dropTable(insertNode.getTableName());
-      catalog.createTable(tableDesc);
+      UpdateTableStatsProto.Builder builder = UpdateTableStatsProto.newBuilder();
+      builder.setTableName(tableDesc.getName());
+      builder.setStats(stats.getProto());
+
+      catalog.updateTableStats(builder.build());
 
       responseBuilder.setTableDesc(tableDesc.getProto());
     } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/95cf4b94/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 6f80171..f92001f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -34,6 +34,8 @@ import org.apache.tajo.QueryId;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoProtos.QueryState;
+import org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
@@ -916,8 +918,11 @@ public class Query implements EventHandler<QueryEvent> {
         finalTable.setStats(stats);
 
         if (insertNode.hasTargetTable()) {
-          catalog.dropTable(insertNode.getTableName());
-          catalog.createTable(finalTable);
+          UpdateTableStatsProto.Builder builder = UpdateTableStatsProto.newBuilder();
+          builder.setTableName(finalTable.getName());
+          builder.setStats(stats.getProto());
+
+          catalog.updateTableStats(builder.build());
         }
 
         query.setResultDesc(finalTable);


[10/10] tajo git commit: Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/tajo into index_support

Posted by ji...@apache.org.
Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/tajo into index_support


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

Branch: refs/heads/index_support
Commit: 761e372ba2b0f4c589ba81c5d9651b55becc1821
Parents: 72600c1 8f68b4b
Author: Jihoon Son <ji...@apache.org>
Authored: Tue Dec 9 10:55:39 2014 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Tue Dec 9 10:55:39 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  18 +-
 pom.xml                                         |   2 +-
 .../tajo/catalog/AbstractCatalogClient.java     |  14 +
 .../src/main/proto/CatalogProtocol.proto        |   1 +
 .../org/apache/tajo/catalog/CatalogService.java |   5 +
 .../src/main/proto/CatalogProtos.proto          |   5 +
 .../tajo/catalog/store/HCatalogStore.java       |   6 +
 .../org/apache/tajo/catalog/CatalogServer.java  |  22 +
 .../tajo/catalog/store/AbstractDBStore.java     |  68 ++
 .../apache/tajo/catalog/store/CatalogStore.java |   4 +-
 .../org/apache/tajo/catalog/store/MemStore.java |  17 +
 .../org/apache/tajo/client/QueryClient.java     |   2 +
 .../org/apache/tajo/client/QueryClientImpl.java |   8 +-
 .../apache/tajo/client/SessionConnection.java   |  33 +
 .../org/apache/tajo/client/TajoClientUtil.java  |   3 +-
 .../org/apache/tajo/jdbc/FetchResultSet.java    |   1 +
 .../apache/tajo/jdbc/TajoMemoryResultSet.java   |   5 +-
 .../org/apache/tajo/jdbc/TajoResultSet.java     |   2 +
 .../org/apache/tajo/jdbc/TajoResultSetBase.java |  38 +-
 .../java/org/apache/tajo/OverridableConf.java   |   7 +-
 .../main/java/org/apache/tajo/SessionVars.java  |   4 +-
 .../java/org/apache/tajo/TajoConstants.java     |   2 +
 .../java/org/apache/tajo/conf/TajoConf.java     |  25 +-
 .../main/java/org/apache/tajo/datum/Datum.java  |   2 +-
 .../org/apache/tajo/datum/DatumFactory.java     |  57 +-
 .../tajo/json/TimeZoneGsonSerdeAdapter.java     |  40 +
 .../apache/tajo/storage/StorageConstants.java   |  22 +-
 .../apache/tajo/util/datetime/DateTimeUtil.java |  23 +-
 .../apache/tajo/datum/TestTimestampDatum.java   |   7 +-
 .../tajo/engine/function/builtin/AvgDouble.java |   3 -
 .../engine/function/datetime/CurrentDate.java   |  20 +-
 .../function/datetime/DatePartFromTime.java     |  20 +-
 .../datetime/DatePartFromTimestamp.java         |  18 +-
 .../function/datetime/ToCharTimestamp.java      |  18 +-
 .../function/datetime/ToTimestampText.java      |  15 +-
 .../tajo/engine/function/string/BTrim.java      |   4 +-
 .../tajo/engine/function/string/LTrim.java      |   4 +-
 .../tajo/engine/function/string/Lpad.java       |   4 +-
 .../tajo/engine/function/string/RTrim.java      |   4 +-
 .../engine/function/string/RegexpReplace.java   |   4 +-
 .../tajo/engine/function/string/Rpad.java       |   4 +-
 .../apache/tajo/engine/json/CoreGsonHelper.java |   2 +
 .../apache/tajo/engine/query/QueryContext.java  |   4 +-
 .../org/apache/tajo/master/GlobalEngine.java    |   8 +-
 .../apache/tajo/master/querymaster/Query.java   |   9 +-
 .../tajo/master/querymaster/QueryMaster.java    |   8 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |   8 +
 .../org/apache/tajo/TajoTestingCluster.java     |   3 +
 .../org/apache/tajo/client/TestTajoClient.java  |  37 +-
 .../apache/tajo/engine/eval/ExprTestBase.java   |  59 +-
 .../tajo/engine/eval/TestIntervalType.java      |  46 +-
 .../tajo/engine/eval/TestSQLDateTimeTypes.java  |  18 +-
 .../tajo/engine/eval/TestSQLExpression.java     |  67 +-
 .../function/TestConditionalExpressions.java    |  16 +-
 .../engine/function/TestDateTimeFunctions.java  | 186 +++--
 .../tajo/engine/query/TestSelectQuery.java      |  50 +-
 .../apache/tajo/engine/query/TestSortQuery.java |  23 +-
 .../org/apache/tajo/jdbc/TestResultSet.java     |  13 +-
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java |   6 -
 .../TestSelectQuery/timezoned/table1.tbl        |   3 +
 .../TestSelectQuery/datetime_table_ddl.sql      |   4 +
 .../datetime_table_timezoned_ddl.sql            |   4 +
 .../TestSelectQuery/testTimezonedTable1.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable2.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable3.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable1.result  |   5 +
 .../TestSelectQuery/testTimezonedTable2.result  |   5 +
 .../TestSelectQuery/testTimezonedTable3.result  |   5 +
 tajo-dist/src/main/bin/tajo                     |   3 +
 tajo-docs/src/main/sphinx/table_management.rst  |   5 +-
 .../src/main/sphinx/table_management/csv.rst    |   4 +-
 .../sphinx/table_management/table_overview.rst  |  98 +++
 .../apache/tajo/jdbc/TajoMetaDataResultSet.java |   2 +
 .../org/apache/tajo/plan/ExprAnnotator.java     |  96 ++-
 .../org/apache/tajo/plan/LogicalPlanner.java    |  28 +-
 .../org/apache/tajo/plan/expr/CastEval.java     |  28 +-
 .../tajo/plan/expr/GeneralFunctionEval.java     |   8 +-
 .../tajo/plan/function/GeneralFunction.java     |   4 +-
 .../plan/serder/EvalTreeProtoDeserializer.java  |   7 +-
 .../plan/serder/EvalTreeProtoSerializer.java    |   3 +
 .../apache/tajo/plan/serder/PlanGsonHelper.java |   2 +
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  21 +
 tajo-plan/src/main/proto/Plan.proto             |   1 +
 tajo-pullserver/pom.xml                         | 146 ++++
 .../tajo/pullserver/FadvisedChunkedFile.java    |  81 ++
 .../tajo/pullserver/FadvisedFileRegion.java     | 170 ++++
 .../FileAccessForbiddenException.java           |  40 +
 .../tajo/pullserver/FileCloseListener.java      |  53 ++
 .../tajo/pullserver/HttpDataServerHandler.java  | 245 ++++++
 .../HttpDataServerPipelineFactory.java          |  56 ++
 .../org/apache/tajo/pullserver/HttpUtil.java    |  69 ++
 .../tajo/pullserver/PullServerAuxService.java   | 654 +++++++++++++++
 .../apache/tajo/pullserver/PullServerUtil.java  |  90 +++
 .../apache/tajo/pullserver/TajoPullServer.java  |  73 ++
 .../tajo/pullserver/TajoPullServerService.java  | 808 +++++++++++++++++++
 .../retriever/AdvancedDataRetriever.java        | 126 +++
 .../pullserver/retriever/DataRetriever.java     |  29 +
 .../retriever/DirectoryRetriever.java           |  56 ++
 .../tajo/pullserver/retriever/FileChunk.java    |  81 ++
 .../pullserver/retriever/RetrieverHandler.java  |  33 +
 .../storage/TextSerializerDeserializer.java     |   7 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |   2 +-
 .../tajo/storage/text/CSVLineSerializer.java    |   2 +-
 .../text/TextFieldSerializerDeserializer.java   |  46 +-
 tajo-yarn-pullserver/pom.xml                    | 146 ----
 .../tajo/pullserver/FadvisedChunkedFile.java    |  81 --
 .../tajo/pullserver/FadvisedFileRegion.java     | 170 ----
 .../FileAccessForbiddenException.java           |  40 -
 .../tajo/pullserver/FileCloseListener.java      |  53 --
 .../tajo/pullserver/HttpDataServerHandler.java  | 245 ------
 .../HttpDataServerPipelineFactory.java          |  56 --
 .../org/apache/tajo/pullserver/HttpUtil.java    |  69 --
 .../tajo/pullserver/PullServerAuxService.java   | 654 ---------------
 .../apache/tajo/pullserver/PullServerUtil.java  |  90 ---
 .../apache/tajo/pullserver/TajoPullServer.java  |  73 --
 .../tajo/pullserver/TajoPullServerService.java  | 808 -------------------
 .../retriever/AdvancedDataRetriever.java        | 126 ---
 .../pullserver/retriever/DataRetriever.java     |  29 -
 .../retriever/DirectoryRetriever.java           |  56 --
 .../tajo/pullserver/retriever/FileChunk.java    |  81 --
 .../pullserver/retriever/RetrieverHandler.java  |  33 -
 121 files changed, 3953 insertions(+), 3187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/761e372b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/761e372b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/761e372b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/761e372b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/761e372b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/761e372b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------


[08/10] tajo git commit: TAJO-1230: Disable ipv6 support on JVM. (Jihun Kang via hyunsik)

Posted by ji...@apache.org.
TAJO-1230: Disable ipv6 support on JVM. (Jihun Kang via hyunsik)

Closes #291


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

Branch: refs/heads/index_support
Commit: 88e5c9e9a5cc067f7cc95899cd21394f2a2243c9
Parents: b5aa780
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 8 18:15:37 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 8 18:15:37 2014 +0900

----------------------------------------------------------------------
 CHANGES                     | 2 ++
 tajo-dist/src/main/bin/tajo | 3 +++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/88e5c9e9/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index c84992b..581c997 100644
--- a/CHANGES
+++ b/CHANGES
@@ -18,6 +18,8 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1230: Disable ipv6 support on JVM. (Jihun Kang via hyunsik)
+
     TAJO-1213: Implement CatalogStore::updateTableStats. (jaehwa)
 
     TAJO-1165: Needs to show error messages on query_executor.jsp. 

http://git-wip-us.apache.org/repos/asf/tajo/blob/88e5c9e9/tajo-dist/src/main/bin/tajo
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo
index d4e99f9..6e054f3 100755
--- a/tajo-dist/src/main/bin/tajo
+++ b/tajo-dist/src/main/bin/tajo
@@ -334,6 +334,9 @@ if [ "$TAJO_POLICYFILE" = "" ]; then
   TAJO_POLICYFILE="tajo-policy.xml"
 fi
 
+# Disable IPv6 Support for network performance
+TAJO_OPTS="$TAJO_OPTS -Djava.net.preferIPv4Stack=true"
+
 # figure out which class to run
 if [ "$COMMAND" = "classpath" ] ; then
   if $cygwin; then