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 2013/12/03 12:50:56 UTC

[01/18] git commit: TAJO-347: Fix bug when to call function with insensitive function name. (DaeMyung Kang via hyunsik)

Updated Branches:
  refs/heads/DAG-execplan 0cbd968ca -> dc24dbc1e


TAJO-347: Fix bug when to call function with insensitive function name. (DaeMyung Kang via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: dc56c8e714c1c930e6b572bfc9b5aed302075d12
Parents: 02fd6cb
Author: Hyunsik Choi <hy...@apache.org>
Authored: Sun Dec 1 12:52:42 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Sun Dec 1 12:54:27 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                                 | 9 +++++++--
 .../src/main/java/org/apache/tajo/algebra/FunctionExpr.java | 4 ++--
 .../engine/function/TestStringOperatorsAndFunctions.java    | 8 ++++++++
 3 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/dc56c8e7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index cd6e9f8..b99cc69 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -52,13 +52,18 @@ Release 0.8.0 - unreleased
 
   BUG FIXES
 
+    TAJO-347: Fix bug when to call function with insensitive function name. 
+    (DaeMyung Kang via hyunsik)
+
     TAJO-345: MergeScanner should support projectable storages. (jihoon)
 
-    TAJO-290: TajoDataType.Type.NULL should be NULL_TYPE. (DaeMyung Kang via jinho)
+    TAJO-290: TajoDataType.Type.NULL should be NULL_TYPE. (DaeMyung Kang via 
+    jinho)
 
     TAJO-332: Invalid row count of CSVScanner. (jinho)
 
-    TAJO-326: In ExecutionBlock, isRoot() and isLeafBlock() return invalid values. (jihoon)
+    TAJO-326: In ExecutionBlock, isRoot() and isLeafBlock() return invalid 
+    values. (jihoon)
 
     TAJO-296: Late registration of Tajo workers. (hyoungjunkim via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/dc56c8e7/tajo-algebra/src/main/java/org/apache/tajo/algebra/FunctionExpr.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/FunctionExpr.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/FunctionExpr.java
index 7c8b33c..008bfc0 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/FunctionExpr.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/FunctionExpr.java
@@ -38,7 +38,7 @@ public class FunctionExpr extends Expr {
     if (type != OpType.Function && type != OpType.CountRowsFunction) {
       throw new IllegalArgumentException("FunctionExpr cannot accept " + type + "type");
     }
-    this.signature = signature;
+    this.signature = signature.toLowerCase();
   }
 
   protected FunctionExpr(OpType type, String signature, Expr [] params) {
@@ -46,7 +46,7 @@ public class FunctionExpr extends Expr {
     if (type != OpType.Function && type != OpType.GeneralSetFunction) {
       throw new IllegalArgumentException("FunctionExpr cannot accept " + type + "type");
     }
-    this.signature = signature;
+    this.signature = signature.toLowerCase();
     setParams(params);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/dc56c8e7/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index 27b9355..f9da0c4 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -53,6 +53,14 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
   }
 
   @Test
+  public void testFunctionCallIngoreCases() throws IOException {
+    testSimpleEval("select ltrim(' trim') ", new String[]{"trim"});
+    testSimpleEval("select LTRIM(' trim') ", new String[]{"trim"});
+    testSimpleEval("select lTRim(' trim') ", new String[]{"trim"});
+    testSimpleEval("select ltrIM(' trim') ", new String[]{"trim"});
+  }
+
+  @Test
   public void testLTrim() throws IOException {
     Schema schema = new Schema();
     schema.addColumn("col1", TEXT);


[07/18] TAJO-317: Improve TajoResourceManager to support more elaborate resource management. (Keuntae Park via jihoon)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestTajoResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestTajoResourceManager.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestTajoResourceManager.java
new file mode 100644
index 0000000..428bf46
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestTajoResourceManager.java
@@ -0,0 +1,390 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol.*;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestTajoResourceManager {
+  private final PrimitiveProtos.BoolProto BOOL_TRUE = PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
+  private final PrimitiveProtos.BoolProto BOOL_FALSE = PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
+  TajoConf tajoConf;
+  TajoWorkerResourceManager tajoWorkerResourceManager;
+  long queryIdTime = System.currentTimeMillis();
+  int numWorkers = 5;
+  float workerDiskSlots = 5.0f;
+  int workerMemoryMB = 512 * 10;
+  WorkerResourceAllocationResponse response;
+
+  private void initResourceManager(boolean queryMasterMode) throws Exception {
+    tajoConf = new org.apache.tajo.conf.TajoConf();
+
+    tajoConf.setFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT, 0.0f);
+    tajoConf.setIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB, 512);
+
+    tajoWorkerResourceManager = new TajoWorkerResourceManager(tajoConf);
+
+    for(int i = 0; i < numWorkers; i++) {
+      ServerStatusProto.System system = ServerStatusProto.System.newBuilder()
+          .setAvailableProcessors(1)
+          .setFreeMemoryMB(workerMemoryMB)
+          .setMaxMemoryMB(workerMemoryMB)
+          .setTotalMemoryMB(workerMemoryMB)
+          .build();
+
+      ServerStatusProto.JvmHeap jvmHeap = ServerStatusProto.JvmHeap.newBuilder()
+          .setFreeHeap(workerMemoryMB)
+          .setMaxHeap(workerMemoryMB)
+          .setTotalHeap(workerMemoryMB)
+          .build();
+
+      ServerStatusProto.Disk disk = ServerStatusProto.Disk.newBuilder()
+          .setAbsolutePath("/")
+          .setFreeSpace(0)
+          .setTotalSpace(0)
+          .setUsableSpace(0)
+          .build();
+
+      List<ServerStatusProto.Disk> disks = new ArrayList<ServerStatusProto.Disk>();
+
+      disks.add(disk);
+
+      ServerStatusProto serverStatus = ServerStatusProto.newBuilder()
+          .setQueryMasterMode(queryMasterMode ? BOOL_TRUE : BOOL_FALSE)
+          .setTaskRunnerMode(BOOL_TRUE)
+          .setDiskSlots(workerDiskSlots)
+          .setMemoryResourceMB(workerMemoryMB)
+          .setJvmHeap(jvmHeap)
+          .setSystem(system)
+          .addAllDisk(disks)
+          .setRunningTaskNum(0)
+          .build();
+
+      TajoHeartbeat tajoHeartbeat = TajoHeartbeat.newBuilder()
+          .setTajoWorkerHost("host" + (i + 1))
+          .setQueryId(QueryIdFactory.newQueryId(queryIdTime, i + 1).getProto())
+          .setTajoQueryMasterPort(21000)
+          .setPeerRpcPort(29000 + i)
+          .setTajoWorkerHttpPort(28080 + i)
+          .setServerStatus(serverStatus)
+          .build();
+
+      tajoWorkerResourceManager.workerHeartbeat(tajoHeartbeat);
+    }
+  }
+
+
+  @Test
+  public void testHeartbeat() throws Exception {
+    initResourceManager(false);
+    assertEquals(numWorkers, tajoWorkerResourceManager.getWorkers().size());
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      assertEquals(workerMemoryMB, eachWorker.getAvailableMemoryMB());
+      assertEquals(workerDiskSlots, eachWorker.getAvailableDiskSlots(), 0);
+    }
+  }
+
+  @Test
+  public void testMemoryResource() throws Exception {
+    initResourceManager(false);
+
+    final int minMemory = 256;
+    final int maxMemory = 512;
+    float diskSlots = 1.0f;
+
+    QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 1);
+    ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(queryId);
+
+    WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder()
+        .setResourceRequestPriority(ResourceRequestPriority.MEMORY)
+        .setNumContainers(60)
+        .setExecutionBlockId(ebId.getProto())
+        .setMaxDiskSlotPerContainer(diskSlots)
+        .setMinDiskSlotPerContainer(diskSlots)
+        .setMinMemoryMBPerContainer(minMemory)
+        .setMaxMemoryMBPerContainer(maxMemory)
+        .build();
+
+    final Object monitor = new Object();
+    final List<YarnProtos.ContainerIdProto> containerIds = new ArrayList<YarnProtos.ContainerIdProto>();
+
+
+    RpcCallback<WorkerResourceAllocationResponse> callBack = new RpcCallback<WorkerResourceAllocationResponse>() {
+
+      @Override
+      public void run(WorkerResourceAllocationResponse response) {
+        TestTajoResourceManager.this.response = response;
+        synchronized(monitor) {
+          monitor.notifyAll();
+        }
+      }
+    };
+
+    tajoWorkerResourceManager.allocateWorkerResources(request, callBack);
+    synchronized(monitor) {
+      monitor.wait();
+    }
+
+
+    // assert after callback
+    int totalUsedMemory = 0;
+    int totalUsedDisks = 0;
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      assertEquals(0, eachWorker.getAvailableMemoryMB());
+      assertEquals(0, eachWorker.getAvailableDiskSlots(), 0);
+      assertEquals(5.0f, eachWorker.getUsedDiskSlots(), 0);
+
+      totalUsedMemory += eachWorker.getUsedMemoryMB();
+      totalUsedDisks += eachWorker.getUsedDiskSlots();
+    }
+
+    assertEquals(workerMemoryMB * numWorkers, totalUsedMemory);
+    assertEquals(workerDiskSlots * numWorkers, totalUsedDisks, 0);
+
+    assertEquals(numWorkers * 10, response.getWorkerAllocatedResourceList().size());
+
+    for(WorkerAllocatedResource eachResource: response.getWorkerAllocatedResourceList()) {
+      assertTrue(
+          eachResource.getAllocatedMemoryMB() >= minMemory &&  eachResource.getAllocatedMemoryMB() <= maxMemory);
+      containerIds.add(eachResource.getContainerId());
+    }
+
+    for(YarnProtos.ContainerIdProto eachContainerId: containerIds) {
+      tajoWorkerResourceManager.releaseWorkerResource(ebId, eachContainerId);
+    }
+
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      assertEquals(workerMemoryMB, eachWorker.getAvailableMemoryMB());
+      assertEquals(0, eachWorker.getUsedMemoryMB());
+
+      assertEquals(workerDiskSlots, eachWorker.getAvailableDiskSlots(), 0);
+      assertEquals(0.0f, eachWorker.getUsedDiskSlots(), 0);
+    }
+  }
+
+  @Test
+  public void testMemoryNotCommensurable() throws Exception {
+    initResourceManager(false);
+
+    final int minMemory = 200;
+    final int maxMemory = 500;
+    float diskSlots = 1.0f;
+
+    QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 2);
+    ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(queryId);
+
+    int requiredContainers = 60;
+
+    int numAllocatedContainers = 0;
+
+    int loopCount = 0;
+    while(true) {
+      WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder()
+          .setResourceRequestPriority(ResourceRequestPriority.MEMORY)
+          .setNumContainers(requiredContainers - numAllocatedContainers)
+          .setExecutionBlockId(ebId.getProto())
+          .setMaxDiskSlotPerContainer(diskSlots)
+          .setMinDiskSlotPerContainer(diskSlots)
+          .setMinMemoryMBPerContainer(minMemory)
+          .setMaxMemoryMBPerContainer(maxMemory)
+          .build();
+
+      final Object monitor = new Object();
+
+      RpcCallback<WorkerResourceAllocationResponse> callBack = new RpcCallback<WorkerResourceAllocationResponse>() {
+        @Override
+        public void run(WorkerResourceAllocationResponse response) {
+          TestTajoResourceManager.this.response = response;
+          synchronized(monitor) {
+            monitor.notifyAll();
+          }
+        }
+      };
+
+      tajoWorkerResourceManager.allocateWorkerResources(request, callBack);
+      synchronized(monitor) {
+        monitor.wait();
+      }
+
+      numAllocatedContainers += TestTajoResourceManager.this.response.getWorkerAllocatedResourceList().size();
+
+      //release resource
+      for(WorkerAllocatedResource eachResource: TestTajoResourceManager.this.response.getWorkerAllocatedResourceList()) {
+        assertTrue(
+            eachResource.getAllocatedMemoryMB() >= minMemory &&  eachResource.getAllocatedMemoryMB() <= maxMemory);
+        tajoWorkerResourceManager.releaseWorkerResource(ebId, eachResource.getContainerId());
+      }
+
+      for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+        assertEquals(0, eachWorker.getUsedMemoryMB());
+        assertEquals(workerMemoryMB, eachWorker.getAvailableMemoryMB());
+
+        assertEquals(0.0f, eachWorker.getUsedDiskSlots(), 0);
+        assertEquals(workerDiskSlots, eachWorker.getAvailableDiskSlots(), 0);
+      }
+
+      loopCount++;
+
+      if(loopCount == 2) {
+        assertEquals(requiredContainers, numAllocatedContainers);
+        break;
+      }
+    }
+
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      assertEquals(0, eachWorker.getUsedMemoryMB());
+      assertEquals(workerMemoryMB, eachWorker.getAvailableMemoryMB());
+
+      assertEquals(0.0f, eachWorker.getUsedDiskSlots(), 0);
+      assertEquals(workerDiskSlots, eachWorker.getAvailableDiskSlots(), 0);
+    }
+  }
+
+  @Test
+  public void testDiskResource() throws Exception {
+    initResourceManager(false);
+    final float minDiskSlots = 1.0f;
+    final float maxDiskSlots = 2.0f;
+    int memoryMB = 256;
+
+    QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 3);
+    ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(queryId);
+
+    WorkerResourceAllocationRequest request = WorkerResourceAllocationRequest.newBuilder()
+        .setResourceRequestPriority(ResourceRequestPriority.DISK)
+        .setNumContainers(60)
+        .setExecutionBlockId(ebId.getProto())
+        .setMaxDiskSlotPerContainer(maxDiskSlots)
+        .setMinDiskSlotPerContainer(minDiskSlots)
+        .setMinMemoryMBPerContainer(memoryMB)
+        .setMaxMemoryMBPerContainer(memoryMB)
+        .build();
+
+    final Object monitor = new Object();
+    final List<YarnProtos.ContainerIdProto> containerIds = new ArrayList<YarnProtos.ContainerIdProto>();
+
+
+    RpcCallback<WorkerResourceAllocationResponse> callBack = new RpcCallback<WorkerResourceAllocationResponse>() {
+
+      @Override
+      public void run(WorkerResourceAllocationResponse response) {
+        TestTajoResourceManager.this.response = response;
+        synchronized(monitor) {
+          monitor.notifyAll();
+        }
+      }
+    };
+
+    tajoWorkerResourceManager.allocateWorkerResources(request, callBack);
+    synchronized(monitor) {
+      monitor.wait();
+    }
+    for(WorkerAllocatedResource eachResource: response.getWorkerAllocatedResourceList()) {
+      assertTrue("AllocatedDiskSlot:" + eachResource.getAllocatedDiskSlots(),
+          eachResource.getAllocatedDiskSlots() >= minDiskSlots &&
+              eachResource.getAllocatedDiskSlots() <= maxDiskSlots);
+      containerIds.add(eachResource.getContainerId());
+    }
+
+    // assert after callback
+    int totalUsedDisks = 0;
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      //each worker allocated 3 container (2 disk slot = 2, 1 disk slot = 1)
+      assertEquals(0, eachWorker.getAvailableDiskSlots(), 0);
+      assertEquals(5.0f, eachWorker.getUsedDiskSlots(), 0);
+      assertEquals(256 * 3, eachWorker.getUsedMemoryMB());
+
+      totalUsedDisks += eachWorker.getUsedDiskSlots();
+    }
+
+    assertEquals(workerDiskSlots * numWorkers, totalUsedDisks, 0);
+
+    assertEquals(numWorkers * 3, response.getWorkerAllocatedResourceList().size());
+
+    for(YarnProtos.ContainerIdProto eachContainerId: containerIds) {
+      tajoWorkerResourceManager.releaseWorkerResource(ebId, eachContainerId);
+    }
+
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      assertEquals(workerMemoryMB, eachWorker.getAvailableMemoryMB());
+      assertEquals(0, eachWorker.getUsedMemoryMB());
+
+      assertEquals(workerDiskSlots, eachWorker.getAvailableDiskSlots(), 0);
+      assertEquals(0.0f, eachWorker.getUsedDiskSlots(), 0);
+    }
+  }
+
+  @Test
+  public void testQueryMasterResource() throws Exception {
+    initResourceManager(true);
+
+    int qmDefaultMemoryMB = tajoConf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB);
+    float qmDefaultDiskSlots = tajoConf.getFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT);
+
+    QueryId queryId = QueryIdFactory.newQueryId(queryIdTime, 4);
+
+    tajoWorkerResourceManager.allocateQueryMaster(queryId);
+
+    // assert after callback
+    int totalUsedMemory = 0;
+    int totalUsedDisks = 0;
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      if(eachWorker.getUsedMemoryMB() > 0) {
+        //worker which allocated querymaster
+        assertEquals(qmDefaultMemoryMB, eachWorker.getUsedMemoryMB());
+        assertEquals(qmDefaultDiskSlots, eachWorker.getUsedDiskSlots(), 0);
+      } else {
+        assertEquals(0, eachWorker.getUsedMemoryMB());
+        assertEquals(0, eachWorker.getUsedDiskSlots(), 0);
+      }
+
+      totalUsedMemory += eachWorker.getUsedMemoryMB();
+      totalUsedDisks += eachWorker.getUsedDiskSlots();
+    }
+
+    assertEquals(qmDefaultMemoryMB, totalUsedMemory);
+    assertEquals(qmDefaultDiskSlots, totalUsedDisks, 0);
+
+    //release
+    tajoWorkerResourceManager.stopQueryMaster(queryId);
+    for(WorkerResource eachWorker: tajoWorkerResourceManager.getWorkers().values()) {
+      assertEquals(0, eachWorker.getUsedMemoryMB());
+      assertEquals(0, eachWorker.getUsedDiskSlots(), 0);
+      totalUsedMemory += eachWorker.getUsedMemoryMB();
+      totalUsedDisks += eachWorker.getUsedDiskSlots();
+    }
+  }
+}


[04/18] git commit: TAJO-351: TAJO-351: Implement reverse(text). (DaeMyung Kang via hyunsik)

Posted by ji...@apache.org.
TAJO-351: TAJO-351: Implement reverse(text). (DaeMyung Kang via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: 14ec0ec133fb0b93ecf9f71628daf80399b0f6a8
Parents: e29b062
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 2 14:27:43 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 2 14:27:43 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../tajo/engine/function/string/Reverse.java    | 48 ++++++++++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  5 ++
 .../TestStringOperatorsAndFunctions.java        | 12 +++++
 4 files changed, 67 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/14ec0ec1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ebb1af0..f051e1d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-351: Implement reverse(text). (DaeMyung Kang via hyunsik)
+
     TAJO-342: Implement strpos(string, substring) function. 
     (hyoungjunkim via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/14ec0ec1/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Reverse.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Reverse.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Reverse.java
new file mode 100644
index 0000000..a08c2d3
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Reverse.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text reverse(string text)
+ */
+public class Reverse extends GeneralFunction {
+  public Reverse() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(new StringBuffer(datum.asChars()).reverse().toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/14ec0ec1/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 5b9c00e..557fd5a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -340,6 +340,11 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataTypeArray(Type.INT4)));
 
     sqlFuncs.add(
+        new FunctionDesc("reverse", Reverse.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
+
+    sqlFuncs.add(
         new FunctionDesc("upper", Upper.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/14ec0ec1/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index 042cc6f..b62b1b8 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -150,6 +150,18 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
   }
 
   @Test
+  public void testReverse() throws IOException {
+    testSimpleEval("select reverse('abcdef') as col1 ", new String[]{"fedcba"});
+    testSimpleEval("select reverse('가') as col1 ", new String[]{"가"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "abc,efg,3.14", "select reverse(col1) || reverse(col2) from table1", new String[]{"cbagfe"});
+  }
+
+  @Test
   public void testUpper() throws IOException {
     testSimpleEval("select upper('abcdef') as col1 ", new String[]{"ABCDEF"});
 


[10/18] git commit: TAJO-357: Fix invalid filename TestMethFunction to TestMathFUnction. (DaeMyung Kang via hyunsik)

Posted by ji...@apache.org.
TAJO-357: Fix invalid filename TestMethFunction to TestMathFUnction. (DaeMyung Kang via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: b3d754e288c0a6004d8e2ade94ad51e690ac0041
Parents: 43ec04a
Author: Hyunsik Choi <hy...@apache.org>
Authored: Tue Dec 3 10:55:07 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Tue Dec 3 10:55:07 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  3 +
 .../java/org/apache/tajo/master/TajoMaster.java |  4 +-
 .../tajo/engine/function/TestMathFunctions.java | 83 ++++++++++++++++++++
 .../tajo/engine/function/TestMethFunction.java  | 83 --------------------
 4 files changed, 87 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b3d754e2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0ad8c3c..b4ab394 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,9 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-357: Fix invalid filename TestMethFunction to TestMathFUnction.
+    (DaeMyung Kang via hyunsik)
+
     TAJO-352: Implement right/left(text, size) function. 
     (DaeMyung Kang via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b3d754e2/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 2fe4213..0ea3c73 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -46,9 +46,7 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.function.Country;
 import org.apache.tajo.engine.function.InCountry;
 import org.apache.tajo.engine.function.builtin.*;
-import org.apache.tajo.engine.function.math.Ceil;
-import org.apache.tajo.engine.function.math.Floor;
-import org.apache.tajo.engine.function.math.Round;
+import org.apache.tajo.engine.function.math.*;
 import org.apache.tajo.engine.function.string.*;
 import org.apache.tajo.master.querymaster.QueryJobManager;
 import org.apache.tajo.master.rm.TajoWorkerResourceManager;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b3d754e2/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
new file mode 100644
index 0000000..6a59b7a
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.FLOAT8;
+
+public class TestMathFunctions extends ExprTestBase {
+  @Test
+  public void testRound() throws IOException {
+    testSimpleEval("select round(5.1) as col1 ", new String[]{"5"});
+    testSimpleEval("select round(5.5) as col1 ", new String[]{"6"});
+    testSimpleEval("select round(5.6) as col1 ", new String[]{"6"});
+
+//    testSimpleEval("select round(-5.1) as col1 ", new String[]{"-5"});
+//    testSimpleEval("select round(-5.5) as col1 ", new String[]{"-6"});
+//    testSimpleEval("select round(-5.6) as col1 ", new String[]{"-6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from table1",
+        new String[]{"2"});
+  }
+
+  @Test
+  public void testFloor() throws IOException {
+    testSimpleEval("select floor(5.1) as col1 ", new String[]{"5"});
+    testSimpleEval("select floor(5.5) as col1 ", new String[]{"5"});
+    testSimpleEval("select floor(5.6) as col1 ", new String[]{"5"});
+//    testSimpleEval("select floor(-5.1) as col1 ", new String[]{"-6"});
+//    testSimpleEval("select floor(-5.6) as col1 ", new String[]{"-6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from table1",
+        new String[]{"1"});
+  }
+
+  @Test
+  public void testCeil() throws IOException {
+    testSimpleEval("select ceil(5.0) as col1 ", new String[]{"5"});
+    testSimpleEval("select ceil(5.1) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceil(5.5) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceil(5.6) as col1 ", new String[]{"6"});
+//    testSimpleEval("select ceil(-5.1) as col1 ", new String[]{"-5"});
+//    testSimpleEval("select ceil(-5.6) as col1 ", new String[]{"-5"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from table1",
+        new String[]{"2"});
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b3d754e2/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java
deleted file mode 100644
index 77b0443..0000000
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.function;
-
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.engine.eval.ExprTestBase;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.tajo.common.TajoDataTypes.Type.FLOAT8;
-
-public class TestMethFunction extends ExprTestBase {
-  @Test
-  public void testRound() throws IOException {
-    testSimpleEval("select round(5.1) as col1 ", new String[]{"5"});
-    testSimpleEval("select round(5.5) as col1 ", new String[]{"6"});
-    testSimpleEval("select round(5.6) as col1 ", new String[]{"6"});
-
-//    testSimpleEval("select round(-5.1) as col1 ", new String[]{"-5"});
-//    testSimpleEval("select round(-5.5) as col1 ", new String[]{"-6"});
-//    testSimpleEval("select round(-5.6) as col1 ", new String[]{"-6"});
-
-    Schema schema = new Schema();
-    schema.addColumn("col1", FLOAT8);
-    schema.addColumn("col2", FLOAT8);
-    schema.addColumn("col3", FLOAT8);
-
-    testEval(schema, "table1", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from table1",
-        new String[]{"2"});
-  }
-
-  @Test
-  public void testFloor() throws IOException {
-    testSimpleEval("select floor(5.1) as col1 ", new String[]{"5"});
-    testSimpleEval("select floor(5.5) as col1 ", new String[]{"5"});
-    testSimpleEval("select floor(5.6) as col1 ", new String[]{"5"});
-//    testSimpleEval("select floor(-5.1) as col1 ", new String[]{"-6"});
-//    testSimpleEval("select floor(-5.6) as col1 ", new String[]{"-6"});
-
-    Schema schema = new Schema();
-    schema.addColumn("col1", FLOAT8);
-    schema.addColumn("col2", FLOAT8);
-    schema.addColumn("col3", FLOAT8);
-
-    testEval(schema, "table1", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from table1",
-        new String[]{"1"});
-  }
-
-  @Test
-  public void testCeil() throws IOException {
-    testSimpleEval("select ceil(5.0) as col1 ", new String[]{"5"});
-    testSimpleEval("select ceil(5.1) as col1 ", new String[]{"6"});
-    testSimpleEval("select ceil(5.5) as col1 ", new String[]{"6"});
-    testSimpleEval("select ceil(5.6) as col1 ", new String[]{"6"});
-//    testSimpleEval("select ceil(-5.1) as col1 ", new String[]{"-5"});
-//    testSimpleEval("select ceil(-5.6) as col1 ", new String[]{"-5"});
-
-    Schema schema = new Schema();
-    schema.addColumn("col1", FLOAT8);
-    schema.addColumn("col2", FLOAT8);
-    schema.addColumn("col3", FLOAT8);
-
-    testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from table1",
-        new String[]{"2"});
-  }
-}


[11/18] git commit: TAJO-348: Implement octet_length(text). (DaeMyung Kang via jaehwa)

Posted by ji...@apache.org.
TAJO-348: Implement octet_length(text). (DaeMyung Kang via jaehwa)


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

Branch: refs/heads/DAG-execplan
Commit: 19c7585cc775764860341fcc9404ee92d80d3b20
Parents: b3d754e
Author: blrunner <jh...@gruter.com>
Authored: Tue Dec 3 16:00:07 2013 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Tue Dec 3 16:00:07 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../engine/function/string/OctetLength.java     | 51 ++++++++++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  5 ++
 .../TestStringOperatorsAndFunctions.java        | 14 ++++++
 4 files changed, 72 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/19c7585c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b4ab394..5967f4b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-348: Implement octet_length(text). (DaeMyung Kang via jaehwa)
+
     TAJO-357: Fix invalid filename TestMethFunction to TestMathFUnction.
     (DaeMyung Kang via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/19c7585c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java
new file mode 100644
index 0000000..345244d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/OctetLength.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT4 octet_length(string text)
+ */
+public class OctetLength  extends GeneralFunction {
+
+  public OctetLength() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt4(datum.asByteArray().length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/19c7585c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 0ea3c73..8139b8e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -431,6 +431,11 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
 
     sqlFuncs.add(
+        new FunctionDesc("octet_length", OctetLength.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT4),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
+
+    sqlFuncs.add(
         new FunctionDesc("substr", Substr.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.INT4, Type.INT4)));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/19c7585c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index c62c494..bbe6939 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -295,6 +295,20 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
   }
 
   @Test
+  public void testOctetLength() throws IOException {
+    testSimpleEval("select octet_length('123456') as col1 ", new String[]{"6"});
+    testSimpleEval("select octet_length('1') as col1 ", new String[]{"1"});
+    testSimpleEval("select octet_length('가') as col1 ", new String[]{"3"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABC,DEF,3.14", "select octet_length(lower(col1) || lower(col2)) from table1",
+        new String[]{"6"});
+  }
+
+  @Test
   public void testSubstr() throws IOException {
     testSimpleEval("select substr('abcdef', 3, 2) as col1 ", new String[]{"cd"});
     testSimpleEval("select substr('abcdef', 3) as col1 ", new String[]{"cdef"});


[08/18] git commit: TAJO-317: Improve TajoResourceManager to support more elaborate resource management. (Keuntae Park via jihoon)

Posted by ji...@apache.org.
TAJO-317: Improve TajoResourceManager to support more elaborate resource management. (Keuntae Park via jihoon)


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

Branch: refs/heads/DAG-execplan
Commit: 528c914f9a133bef79df07017cfa424c9fab4412
Parents: 778c01f
Author: Jihoon Son <ji...@apache.org>
Authored: Mon Dec 2 17:16:01 2013 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Mon Dec 2 17:16:01 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../java/org/apache/tajo/conf/TajoConf.java     |  14 +-
 .../apache/tajo/master/TajoContainerProxy.java  |  38 +-
 .../apache/tajo/master/TajoMasterService.java   |  25 +-
 .../tajo/master/querymaster/SubQuery.java       |   9 +-
 .../tajo/master/rm/TajoWorkerContainerId.java   |  41 ++
 .../master/rm/TajoWorkerResourceManager.java    | 358 +++++++++++++----
 .../apache/tajo/master/rm/WorkerResource.java   | 115 +++---
 .../tajo/master/rm/WorkerResourceManager.java   |   6 +-
 .../tajo/master/rm/YarnTajoResourceManager.java |  28 +-
 .../apache/tajo/util/ApplicationIdUtils.java    |   3 +
 .../apache/tajo/worker/ResourceAllocator.java   |   3 +-
 .../tajo/worker/TajoResourceAllocator.java      |  68 ++--
 .../java/org/apache/tajo/worker/TajoWorker.java |  86 ++--
 .../main/java/org/apache/tajo/worker/Task.java  |   6 +-
 .../tajo/worker/YarnResourceAllocator.java      |   4 +-
 .../src/main/proto/TajoMasterProtocol.proto     |  64 ++-
 .../main/resources/webapps/admin/cluster.jsp    |  31 +-
 .../src/main/resources/webapps/admin/index.jsp  |  22 +-
 .../org/apache/tajo/TajoTestingCluster.java     |   3 +
 .../tajo/master/TestTajoResourceManager.java    | 390 +++++++++++++++++++
 21 files changed, 1013 insertions(+), 303 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index caadc0c..1646d4c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -34,6 +34,8 @@ Release 0.8.0 - unreleased
 
   IMPROVEMENTS
 
+    TAJO-317: Improve TajoResourceManager to support more elaborate resource management. (Keuntae Park via jihoon)
+
     TAJO-314: Make TaskScheduler be pluggable. (jihoon)
 
     TAJO-325: QueryState.NEW and QueryState.INIT should be combined into one

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/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 eba6eaf..f89838f 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
@@ -81,6 +81,10 @@ public class TajoConf extends YarnConfiguration {
     TAJO_MASTER_CLIENT_RPC_ADDRESS("tajo.master.client-rpc.address", "localhost:26002"),
     TAJO_MASTER_INFO_ADDRESS("tajo.master.info-http.address", "0.0.0.0:26080"),
 
+    // QueryMaster resource
+    TAJO_QUERYMASTER_DISK_SLOT("tajo.qm.resource.disk.slots", 0.0f),
+    TAJO_QUERYMASTER_MEMORY_MB("tajo.qm.resource.memory-mb", 512),
+
     // Tajo Worker Service Addresses
     WORKER_INFO_ADDRESS("tajo.worker.info-http.address", "0.0.0.0:28080"),
     WORKER_QM_INFO_ADDRESS("tajo.worker.qm-info-http.address", "0.0.0.0:28081"),
@@ -93,8 +97,8 @@ public class TajoConf extends YarnConfiguration {
 
     // Tajo Worker Resources
     WORKER_RESOURCE_AVAILABLE_CPU_CORES("tajo.worker.resource.cpu-cores", 1),
-    WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 1024),
-    WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1),
+    WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 512),
+    WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1.0f),
     WORKER_EXECUTION_MAX_SLOTS("tajo.worker.parallel-execution.max-num", 2),
 
     // Tajo Worker Dedicated Resources
@@ -186,6 +190,12 @@ public class TajoConf extends YarnConfiguration {
     // Hive Configuration
     //////////////////////////////////
     HIVE_QUERY_MODE("tajo.hive.query.mode", false),
+
+    //////////////////////////////////
+    // Task Configuration
+    TASK_DEFAULT_MEMORY("tajo.task.memory-slot-mb.default", 512),
+    TASK_DEFAULT_DISK("tajo.task.disk-slot.default", 1.0f)
+    //////////////////////////////////
     ;
 
     public final String varname;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
index d542390..ce5f401 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
@@ -21,7 +21,9 @@ package org.apache.tajo.master;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
@@ -29,7 +31,7 @@ import org.apache.tajo.master.event.QueryEvent;
 import org.apache.tajo.master.event.QueryEventType;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.master.rm.TajoWorkerContainer;
-import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.master.rm.TajoWorkerContainerId;
 import org.apache.tajo.rpc.AsyncRpcClient;
 import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.NullCallback;
@@ -121,7 +123,8 @@ public class TajoContainerProxy extends ContainerProxy {
       this.state = ContainerState.KILLED_BEFORE_LAUNCH;
     } else {
       try {
-        releaseWorkerResource(context, executionBlockId, ((TajoWorkerContainer)container).getWorkerResource());
+        TajoWorkerContainer tajoWorkerContainer = ((TajoWorkerContainer)container);
+        releaseWorkerResource(context, executionBlockId, tajoWorkerContainer.getId());
         context.getResourceAllocator().removeContainer(containerID);
         this.state = ContainerState.DONE;
       } catch (Throwable t) {
@@ -138,29 +141,21 @@ public class TajoContainerProxy extends ContainerProxy {
 
   public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext context,
                                            ExecutionBlockId executionBlockId,
-                                           WorkerResource workerResource) throws Exception {
-    List<WorkerResource> workerResources = new ArrayList<WorkerResource>();
-    workerResources.add(workerResource);
+                                           ContainerId containerId) throws Exception {
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
 
-    releaseWorkerResource(context, executionBlockId, workerResources);
+    releaseWorkerResource(context, executionBlockId, containerIds);
   }
 
   public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext context,
                                            ExecutionBlockId executionBlockId,
-                                           List<WorkerResource> workerResources) throws Exception {
-    List<TajoMasterProtocol.WorkerResourceProto> workerResourceProtos =
-        new ArrayList<TajoMasterProtocol.WorkerResourceProto>();
-
-    for(WorkerResource eahWorkerResource: workerResources) {
-      workerResourceProtos.add(TajoMasterProtocol.WorkerResourceProto.newBuilder()
-          .setHost(eahWorkerResource.getAllocatedHost())
-          .setQueryMasterPort(eahWorkerResource.getQueryMasterPort())
-          .setPeerRpcPort(eahWorkerResource.getPeerRpcPort())
-          .setExecutionBlockId(executionBlockId.getProto())
-          .setMemoryMBSlots(eahWorkerResource.getMemoryMBSlots())
-          .setDiskSlots(eahWorkerResource.getDiskSlots())
-          .build()
-      );
+                                           List<ContainerId> containerIds) throws Exception {
+    List<YarnProtos.ContainerIdProto> containerIdProtos =
+        new ArrayList<YarnProtos.ContainerIdProto>();
+
+    for(ContainerId eachContainerId: containerIds) {
+      containerIdProtos.add(TajoWorkerContainerId.getContainerIdProto(eachContainerId));
     }
 
     RpcConnectionPool connPool = RpcConnectionPool.getPool(context.getConf());
@@ -171,7 +166,8 @@ public class TajoContainerProxy extends ContainerProxy {
         TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
         masterClientService.releaseWorkerResource(null,
           TajoMasterProtocol.WorkerResourceReleaseRequest.newBuilder()
-              .addAllWorkerResources(workerResourceProtos)
+              .setExecutionBlockId(executionBlockId.getProto())
+              .addAllContainerIds(containerIdProtos)
               .build(),
           NullCallback.get());
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
index cf193de..c213dd5 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
@@ -22,13 +22,14 @@ import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.master.querymaster.QueryJobManager;
-import org.apache.tajo.master.rm.WorkerResource;
 import org.apache.tajo.rpc.AsyncRpcServer;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
@@ -118,8 +119,7 @@ public class TajoMasterService extends AbstractService {
         builder.setResponseCommand(command);
       }
 
-      builder.setNumClusterNodes(context.getResourceManager().getWorkers().size());
-      builder.setNumClusterSlots(context.getResourceManager().getNumClusterSlots());
+      builder.setClusterResourceSummary(context.getResourceManager().getClusterResourceSummary());
       done.run(builder.build());
     }
 
@@ -135,20 +135,11 @@ public class TajoMasterService extends AbstractService {
     public void releaseWorkerResource(RpcController controller,
                                            TajoMasterProtocol.WorkerResourceReleaseRequest request,
                                            RpcCallback<PrimitiveProtos.BoolProto> done) {
-      List<TajoMasterProtocol.WorkerResourceProto> workerResources = request.getWorkerResourcesList();
-      for(TajoMasterProtocol.WorkerResourceProto eachWorkerResource: workerResources) {
-        WorkerResource workerResource = new WorkerResource();
-        workerResource.setAllocatedHost(eachWorkerResource.getHost());
-
-        workerResource.setPeerRpcPort(eachWorkerResource.getPeerRpcPort());
-        workerResource.setQueryMasterPort(eachWorkerResource.getQueryMasterPort());
-        workerResource.setMemoryMBSlots(eachWorkerResource.getMemoryMBSlots());
-        workerResource.setDiskSlots(eachWorkerResource.getDiskSlots());
-
-        LOG.info("releaseWorkerResource:" + workerResource);
-        context.getResourceManager().releaseWorkerResource(
-            new QueryId(eachWorkerResource.getExecutionBlockId().getQueryId()),
-            workerResource);
+      List<YarnProtos.ContainerIdProto> containerIds = request.getContainerIdsList();
+      ExecutionBlockId ebId = new ExecutionBlockId(request.getExecutionBlockId());
+
+      for(YarnProtos.ContainerIdProto eachContainer: containerIds) {
+        context.getResourceManager().releaseWorkerResource(ebId, eachContainer);
       }
       done.run(BOOL_TRUE);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 4aa3866..70bde5c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -611,13 +611,18 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
       ExecutionBlock execBlock = subQuery.getBlock();
       QueryUnit [] tasks = subQuery.getQueryUnits();
 
+      //TODO consider disk slot
+      int requiredMemoryMBPerTask = 512;
+
       int numRequest = subQuery.getContext().getResourceAllocator().calculateNumRequestContainers(
-          subQuery.getContext().getQueryMasterContext().getWorkerContext(), tasks.length
+          subQuery.getContext().getQueryMasterContext().getWorkerContext(),
+          tasks.length,
+          requiredMemoryMBPerTask
       );
 
       final Resource resource = Records.newRecord(Resource.class);
 
-      resource.setMemory(2000);
+      resource.setMemory(requiredMemoryMBPerTask);
 
       LOG.info("Request Container for " + subQuery.getId() + " containers=" + numRequest);
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
index f104637..2d7c1c3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
@@ -20,6 +20,7 @@ package org.apache.tajo.master.rm;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 
 public class TajoWorkerContainerId extends ContainerId {
   ApplicationAttemptId applicationAttemptId;
@@ -44,4 +45,44 @@ public class TajoWorkerContainerId extends ContainerId {
   public void setId(int id) {
     this.id = id;
   }
+
+  public YarnProtos.ContainerIdProto getProto() {
+    YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder()
+        .setClusterTimestamp(applicationAttemptId.getApplicationId().getClusterTimestamp())
+        .setId(applicationAttemptId.getApplicationId().getId())
+        .build();
+
+    YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder()
+        .setAttemptId(applicationAttemptId.getAttemptId())
+        .setApplicationId(appIdProto)
+        .build();
+
+    return YarnProtos.ContainerIdProto.newBuilder()
+        .setAppAttemptId(attemptIdProto)
+        .setAppId(appIdProto)
+        .setId(id)
+        .build();
+  }
+
+  public static YarnProtos.ContainerIdProto getContainerIdProto(ContainerId containerId) {
+    if(containerId instanceof TajoWorkerContainerId) {
+      return ((TajoWorkerContainerId)containerId).getProto();
+    } else {
+      YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder()
+          .setClusterTimestamp(containerId.getApplicationAttemptId().getApplicationId().getClusterTimestamp())
+          .setId(containerId.getApplicationAttemptId().getApplicationId().getId())
+          .build();
+
+      YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder()
+          .setAttemptId(containerId.getApplicationAttemptId().getAttemptId())
+          .setApplicationId(appIdProto)
+          .build();
+
+      return YarnProtos.ContainerIdProto.newBuilder()
+          .setAppAttemptId(attemptIdProto)
+          .setAppId(appIdProto)
+          .setId(containerId.getId())
+          .build();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
index 7ffc563..1485ffe 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -22,6 +22,8 @@ import com.google.protobuf.RpcCallback;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
@@ -30,17 +32,20 @@ import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.querymaster.QueryInProgress;
 import org.apache.tajo.master.querymaster.QueryJobEvent;
-import org.apache.tajo.worker.TajoWorker;
+import org.apache.tajo.util.ApplicationIdUtils;
 
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 public class TajoWorkerResourceManager implements WorkerResourceManager {
   private static final Log LOG = LogFactory.getLog(TajoWorkerResourceManager.class);
 
+  static AtomicInteger containerIdSeq = new AtomicInteger(0);
+
   private TajoMaster.MasterContext masterContext;
 
   //all workers(include querymaster)
@@ -59,22 +64,46 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
 
   private final Object workerResourceLock = new Object();
 
-  private final String queryIdSeed;
+  private String queryIdSeed;
 
   private WorkerResourceAllocationThread workerResourceAllocator;
 
   private WorkerMonitorThread workerMonitor;
 
-  private final BlockingQueue<WorkerResourceRequest> requestQueue;
+  private BlockingQueue<WorkerResourceRequest> requestQueue;
 
-  private final List<WorkerResourceRequest> reAllocationList;
+  private List<WorkerResourceRequest> reAllocationList;
 
   private AtomicBoolean stopped = new AtomicBoolean(false);
 
+  private float queryMasterDefaultDiskSlot;
+
+  private int queryMasterDefaultMemoryMB;
+
+  private TajoConf tajoConf;
+
+  private Map<YarnProtos.ContainerIdProto, AllocatedWorkerResource> allocatedResourceMap =
+      new HashMap<YarnProtos.ContainerIdProto, AllocatedWorkerResource>();
+
   public TajoWorkerResourceManager(TajoMaster.MasterContext masterContext) {
     this.masterContext = masterContext;
+    init(masterContext.getConf());
+  }
+
+  public TajoWorkerResourceManager(TajoConf tajoConf) {
+    init(tajoConf);
+  }
+
+  private void init(TajoConf tajoConf) {
+    this.tajoConf = tajoConf;
     this.queryIdSeed = String.valueOf(System.currentTimeMillis());
 
+    this.queryMasterDefaultDiskSlot =
+        tajoConf.getFloatVar(TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT);
+
+    this.queryMasterDefaultMemoryMB =
+        tajoConf.getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB);
+
     requestQueue = new LinkedBlockingDeque<WorkerResourceRequest>();
     reAllocationList = new ArrayList<WorkerResourceRequest>();
 
@@ -93,15 +122,41 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
     return Collections.unmodifiableSet(liveQueryMasterWorkerResources);
   }
 
-  public int getNumClusterSlots() {
-    int numSlots = 0;
+  @Override
+  public TajoMasterProtocol.ClusterResourceSummary getClusterResourceSummary() {
+    int totalDiskSlots = 0;
+    int totalCpuCoreSlots = 0;
+    int totalMemoryMB = 0;
+
+    int totalAvailableDiskSlots = 0;
+    int totalAvailableCpuCoreSlots = 0;
+    int totalAvailableMemoryMB = 0;
+
     synchronized(workerResourceLock) {
       for(String eachWorker: liveWorkerResources) {
-        numSlots += allWorkerResourceMap.get(eachWorker).getSlots();
+        WorkerResource worker = allWorkerResourceMap.get(eachWorker);
+        if(worker != null) {
+          totalMemoryMB += worker.getMemoryMB();
+          totalAvailableMemoryMB += worker.getAvailableMemoryMB();
+
+          totalDiskSlots += worker.getDiskSlots();
+          totalAvailableDiskSlots += worker.getAvailableDiskSlots();
+
+          totalCpuCoreSlots += worker.getCpuCoreSlots();
+          totalAvailableCpuCoreSlots += worker.getAvailableCpuCoreSlots();
+        }
       }
     }
 
-    return numSlots;
+    return TajoMasterProtocol.ClusterResourceSummary.newBuilder()
+            .setNumWorkers(liveWorkerResources.size())
+            .setTotalCpuCoreSlots(totalCpuCoreSlots)
+            .setTotalDiskSlots(totalDiskSlots)
+            .setTotalMemoryMB(totalMemoryMB)
+            .setTotalAvailableCpuCoreSlots(totalAvailableCpuCoreSlots)
+            .setTotalAvailableDiskSlots(totalAvailableDiskSlots)
+            .setTotalAvailableMemoryMB(totalAvailableMemoryMB)
+            .build();
   }
 
   @Override
@@ -120,9 +175,13 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
 
   @Override
   public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress) {
+    return allocateQueryMaster(queryInProgress.getQueryId());
+  }
+
+  public WorkerResource allocateQueryMaster(QueryId queryId) {
     synchronized(workerResourceLock) {
       if(liveQueryMasterWorkerResources.size() == 0) {
-        LOG.warn("No available resource for querymaster:" + queryInProgress.getQueryId());
+        LOG.warn("No available resource for querymaster:" + queryId);
         return null;
       }
       WorkerResource queryMasterWorker = null;
@@ -137,9 +196,9 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
       if(queryMasterWorker == null) {
         return null;
       }
-      queryMasterWorker.addNumQueryMasterTask();
-      queryMasterMap.put(queryInProgress.getQueryId(), queryMasterWorker);
-      LOG.info(queryInProgress.getQueryId() + "'s QueryMaster is " + queryMasterWorker);
+      queryMasterWorker.addNumQueryMasterTask(queryMasterDefaultDiskSlot, queryMasterDefaultMemoryMB);
+      queryMasterMap.put(queryId, queryMasterWorker);
+      LOG.info(queryId + "'s QueryMaster is " + queryMasterWorker);
       return queryMasterWorker;
     }
   }
@@ -152,15 +211,23 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
     }
 
     if(queryMasterWorkerResource != null) {
-      startQueryMaster(queryInProgress.getQueryId(), queryMasterWorkerResource);
+      AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource();
+      allocatedWorkerResource.workerResource = queryMasterWorkerResource;
+      allocatedWorkerResource.allocatedMemoryMB = queryMasterDefaultMemoryMB;
+      allocatedWorkerResource.allocatedDiskSlots = queryMasterDefaultDiskSlot;
+
+      startQueryMaster(queryInProgress.getQueryId(), allocatedWorkerResource);
     } else {
       //add queue
       TajoMasterProtocol.WorkerResourceAllocationRequest request =
           TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
-            .setMemoryMBSlots(1)
-            .setDiskSlots(1)
             .setExecutionBlockId(QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0).getProto())
-            .setNumWorks(1)
+            .setNumContainers(1)
+            .setMinMemoryMBPerContainer(queryMasterDefaultMemoryMB)
+            .setMaxMemoryMBPerContainer(queryMasterDefaultMemoryMB)
+            .setMinDiskSlotPerContainer(queryMasterDefaultDiskSlot)
+            .setMaxDiskSlotPerContainer(queryMasterDefaultDiskSlot)
+            .setResourceRequestPriority(TajoMasterProtocol.ResourceRequestPriority.MEMORY)
             .build();
       try {
         requestQueue.put(new WorkerResourceRequest(queryInProgress.getQueryId(), true, request, null));
@@ -169,13 +236,13 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
     }
   }
 
-  private void startQueryMaster(QueryId queryId, WorkerResource workResource) {
+  private void startQueryMaster(QueryId queryId, AllocatedWorkerResource workResource) {
     QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
     if(queryInProgress == null) {
       LOG.warn("No QueryInProgress while starting  QueryMaster:" + queryId);
       return;
     }
-    queryInProgress.getQueryInfo().setQueryMasterResource(workResource);
+    queryInProgress.getQueryInfo().setQueryMasterResource(workResource.workerResource);
 
     //fire QueryJobStart event
     queryInProgress.getEventHandler().handle(
@@ -205,7 +272,7 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
 
     @Override
     public void run() {
-      heartbeatTimeout = masterContext.getConf().getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_TIMEOUT);
+      heartbeatTimeout = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_TIMEOUT);
       LOG.info("WorkerMonitor start");
       while(!stopped.get()) {
         try {
@@ -272,6 +339,12 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
     }
   }
 
+  class AllocatedWorkerResource {
+    WorkerResource workerResource;
+    int allocatedMemoryMB;
+    float allocatedDiskSlots;
+  }
+
   class WorkerResourceAllocationThread extends Thread {
     @Override
     public void run() {
@@ -283,36 +356,56 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
           if (LOG.isDebugEnabled()) {
             LOG.debug("allocateWorkerResources:" +
                 (new ExecutionBlockId(resourceRequest.request.getExecutionBlockId())) +
-                ", required:" + resourceRequest.request.getNumWorks() +
+                ", requiredMemory:" + resourceRequest.request.getMinMemoryMBPerContainer() +
+                "~" + resourceRequest.request.getMaxMemoryMBPerContainer() +
+                ", requiredContainers:" + resourceRequest.request.getNumContainers() +
+                ", requiredDiskSlots:" + resourceRequest.request.getMinDiskSlotPerContainer() +
+                "~" + resourceRequest.request.getMaxDiskSlotPerContainer() +
                 ", queryMasterRequest=" + resourceRequest.queryMasterRequest +
                 ", liveWorkers=" + liveWorkerResources.size());
           }
 
-          List<WorkerResource> workerResources = chooseWorkers(
-              resourceRequest.request.getMemoryMBSlots(),
-              resourceRequest.request.getDiskSlots(),
-              resourceRequest.request.getNumWorks());
-
-          LOG.debug("allocateWorkerResources: allocated:" + workerResources.size());
+          List<AllocatedWorkerResource> allocatedWorkerResources = chooseWorkers(resourceRequest);
 
-          if(workerResources.size() > 0) {
+          if(allocatedWorkerResources.size() > 0) {
             if(resourceRequest.queryMasterRequest) {
-              startQueryMaster(resourceRequest.queryId, workerResources.get(0));
+              startQueryMaster(resourceRequest.queryId, allocatedWorkerResources.get(0));
             } else {
-              List<TajoMasterProtocol.WorkerAllocatedResource> workerHosts =
+              List<TajoMasterProtocol.WorkerAllocatedResource> allocatedResources =
                   new ArrayList<TajoMasterProtocol.WorkerAllocatedResource>();
 
-              for(WorkerResource eachWorker: workerResources) {
-                workerHosts.add(TajoMasterProtocol.WorkerAllocatedResource.newBuilder()
-                    .setWorkerHost(eachWorker.getAllocatedHost())
-                    .setQueryMasterPort(eachWorker.getQueryMasterPort())
-                    .setPeerRpcPort(eachWorker.getPeerRpcPort())
-                    .setWorkerPullServerPort(eachWorker.getPullServerPort())
+              for(AllocatedWorkerResource eachWorker: allocatedWorkerResources) {
+                NodeIdPBImpl nodeId = new NodeIdPBImpl();
+
+                nodeId.setHost(eachWorker.workerResource.getAllocatedHost());
+                nodeId.setPort(eachWorker.workerResource.getPeerRpcPort());
+
+                TajoWorkerContainerId containerId = new TajoWorkerContainerId();
+
+                containerId.setApplicationAttemptId(
+                    ApplicationIdUtils.createApplicationAttemptId(resourceRequest.queryId));
+                containerId.setId(containerIdSeq.incrementAndGet());
+
+                YarnProtos.ContainerIdProto containerIdProto = containerId.getProto();
+                allocatedResources.add(TajoMasterProtocol.WorkerAllocatedResource.newBuilder()
+                    .setContainerId(containerIdProto)
+                    .setNodeId(nodeId.toString())
+                    .setWorkerHost(eachWorker.workerResource.getAllocatedHost())
+                    .setQueryMasterPort(eachWorker.workerResource.getQueryMasterPort())
+                    .setPeerRpcPort(eachWorker.workerResource.getPeerRpcPort())
+                    .setWorkerPullServerPort(eachWorker.workerResource.getPullServerPort())
+                    .setAllocatedMemoryMB(eachWorker.allocatedMemoryMB)
+                    .setAllocatedDiskSlots(eachWorker.allocatedDiskSlots)
                     .build());
+
+                synchronized(workerResourceLock) {
+                  allocatedResourceMap.put(containerIdProto, eachWorker);
+                }
               }
+
               resourceRequest.callBack.run(TajoMasterProtocol.WorkerResourceAllocationResponse.newBuilder()
                   .setExecutionBlockId(resourceRequest.request.getExecutionBlockId())
-                  .addAllWorkerAllocatedResource(workerHosts)
+                  .addAllWorkerAllocatedResource(allocatedResources)
                   .build()
               );
             }
@@ -335,54 +428,179 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
     }
   }
 
-  private List<WorkerResource> chooseWorkers(int requiredMemoryMBSlots, int requiredDiskSlots,
-                                             int numWorkerSlots) {
-    List<WorkerResource> selectedWorkers = new ArrayList<WorkerResource>();
+  private List<AllocatedWorkerResource> chooseWorkers(WorkerResourceRequest resourceRequest) {
+    List<AllocatedWorkerResource> selectedWorkers = new ArrayList<AllocatedWorkerResource>();
 
-    int selectedCount = 0;
+    int allocatedResources = 0;
+
+    if(resourceRequest.queryMasterRequest) {
+      WorkerResource worker = allocateQueryMaster(resourceRequest.queryId);
+      if(worker != null) {
+        AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource();
+        allocatedWorkerResource.workerResource = worker;
+        allocatedWorkerResource.allocatedDiskSlots = queryMasterDefaultDiskSlot;
+        allocatedWorkerResource.allocatedMemoryMB = queryMasterDefaultMemoryMB;
+        selectedWorkers.add(allocatedWorkerResource);
+
+        return selectedWorkers;
+      }
+    }
+
+    TajoMasterProtocol.ResourceRequestPriority resourceRequestPriority
+        = resourceRequest.request.getResourceRequestPriority();
+
+    if(resourceRequestPriority == TajoMasterProtocol.ResourceRequestPriority.MEMORY) {
+      synchronized(workerResourceLock) {
+        List<String> randomWorkers = new ArrayList<String>(liveWorkerResources);
+        Collections.shuffle(randomWorkers);
+
+        int numContainers = resourceRequest.request.getNumContainers();
+        int minMemoryMB = resourceRequest.request.getMinMemoryMBPerContainer();
+        int maxMemoryMB = resourceRequest.request.getMaxMemoryMBPerContainer();
+        float diskSlot = Math.max(resourceRequest.request.getMaxDiskSlotPerContainer(),
+            resourceRequest.request.getMinDiskSlotPerContainer());
+
+        int liveWorkerSize = randomWorkers.size();
+        Set<String> insufficientWorkers = new HashSet<String>();
+        boolean stop = false;
+        boolean checkMax = true;
+        while(!stop) {
+          if(allocatedResources >= numContainers) {
+            break;
+          }
+
+          if(insufficientWorkers.size() >= liveWorkerSize) {
+            if(!checkMax) {
+              break;
+            }
+            insufficientWorkers.clear();
+            checkMax = false;
+          }
+          int compareAvailableMemory = checkMax ? maxMemoryMB : minMemoryMB;
+
+          for(String eachWorker: randomWorkers) {
+            if(allocatedResources >= numContainers) {
+              stop = true;
+              break;
+            }
+
+            if(insufficientWorkers.size() >= liveWorkerSize) {
+              break;
+            }
 
-    synchronized(workerResourceLock) {
-      List<String> randomWorkers = new ArrayList<String>(liveWorkerResources);
-      Collections.shuffle(randomWorkers);
-      int liveWorkerSize = randomWorkers.size();
-      Set<String> insufficientWorkers = new HashSet<String>();
-      boolean stop = false;
-      while(!stop) {
-        if(insufficientWorkers.size() >= liveWorkerSize || selectedCount >= numWorkerSlots) {
-          break;
-        }
-        for(String eachWorker: randomWorkers) {
-          if(insufficientWorkers.size() >= liveWorkerSize || selectedCount >= numWorkerSlots) {
-            stop = true;
-          } else {
             WorkerResource workerResource = allWorkerResourceMap.get(eachWorker);
-            if(workerResource.getAvailableMemoryMBSlots() >= requiredMemoryMBSlots) {
-              workerResource.addUsedMemoryMBSlots(requiredMemoryMBSlots);
-              //workerResource.addUsedDiskSlots(requiredDiskSlots);
-              selectedWorkers.add(workerResource);
-              selectedCount++;
+            if(workerResource.getAvailableMemoryMB() >= compareAvailableMemory) {
+              int workerMemory;
+              if(workerResource.getAvailableMemoryMB() >= maxMemoryMB) {
+                workerMemory = maxMemoryMB;
+              } else {
+                workerMemory = workerResource.getAvailableMemoryMB();
+              }
+              AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource();
+              allocatedWorkerResource.workerResource = workerResource;
+              allocatedWorkerResource.allocatedMemoryMB = workerMemory;
+              if(workerResource.getAvailableDiskSlots() >= diskSlot) {
+                allocatedWorkerResource.allocatedDiskSlots = diskSlot;
+              } else {
+                allocatedWorkerResource.allocatedDiskSlots = workerResource.getAvailableDiskSlots();
+              }
+
+              workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots,
+                  allocatedWorkerResource.allocatedMemoryMB);
+
+              selectedWorkers.add(allocatedWorkerResource);
+
+              allocatedResources++;
             } else {
               insufficientWorkers.add(eachWorker);
             }
           }
         }
-        if(!stop) {
-          for(String eachWorker: insufficientWorkers) {
-            randomWorkers.remove(eachWorker);
+      }
+    } else {
+      synchronized(workerResourceLock) {
+        List<String> randomWorkers = new ArrayList<String>(liveWorkerResources);
+        Collections.shuffle(randomWorkers);
+
+        int numContainers = resourceRequest.request.getNumContainers();
+        float minDiskSlots = resourceRequest.request.getMinDiskSlotPerContainer();
+        float maxDiskSlots = resourceRequest.request.getMaxDiskSlotPerContainer();
+        int memoryMB = Math.max(resourceRequest.request.getMaxMemoryMBPerContainer(),
+            resourceRequest.request.getMinMemoryMBPerContainer());
+
+        int liveWorkerSize = randomWorkers.size();
+        Set<String> insufficientWorkers = new HashSet<String>();
+        boolean stop = false;
+        boolean checkMax = true;
+        while(!stop) {
+          if(allocatedResources >= numContainers) {
+            break;
+          }
+
+          if(insufficientWorkers.size() >= liveWorkerSize) {
+            if(!checkMax) {
+              break;
+            }
+            insufficientWorkers.clear();
+            checkMax = false;
+          }
+          float compareAvailableDisk = checkMax ? maxDiskSlots : minDiskSlots;
+
+          for(String eachWorker: randomWorkers) {
+            if(allocatedResources >= numContainers) {
+              stop = true;
+              break;
+            }
+
+            if(insufficientWorkers.size() >= liveWorkerSize) {
+              break;
+            }
+
+            WorkerResource workerResource = allWorkerResourceMap.get(eachWorker);
+            if(workerResource.getAvailableDiskSlots() >= compareAvailableDisk) {
+              float workerDiskSlots;
+              if(workerResource.getAvailableDiskSlots() >= maxDiskSlots) {
+                workerDiskSlots = maxDiskSlots;
+              } else {
+                workerDiskSlots = workerResource.getAvailableDiskSlots();
+              }
+              AllocatedWorkerResource allocatedWorkerResource = new AllocatedWorkerResource();
+              allocatedWorkerResource.workerResource = workerResource;
+              allocatedWorkerResource.allocatedDiskSlots = workerDiskSlots;
+
+              if(workerResource.getAvailableMemoryMB() >= memoryMB) {
+                allocatedWorkerResource.allocatedMemoryMB = memoryMB;
+              } else {
+                allocatedWorkerResource.allocatedMemoryMB = workerResource.getAvailableMemoryMB();
+              }
+              workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots,
+                  allocatedWorkerResource.allocatedMemoryMB);
+
+              selectedWorkers.add(allocatedWorkerResource);
+
+              allocatedResources++;
+            } else {
+              insufficientWorkers.add(eachWorker);
+            }
           }
         }
       }
     }
-
     return selectedWorkers;
   }
 
   @Override
-  public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource) {
+  public void releaseWorkerResource(ExecutionBlockId ebId, YarnProtos.ContainerIdProto containerId) {
     synchronized(workerResourceLock) {
-      WorkerResource managedWorkerResource = allWorkerResourceMap.get(workerResource.getId());
-      if(managedWorkerResource != null) {
-        managedWorkerResource.releaseResource(workerResource);
+      AllocatedWorkerResource allocatedWorkerResource = allocatedResourceMap.get(containerId);
+      if(allocatedWorkerResource != null) {
+        LOG.info("Release Resource:" + ebId + "," +
+            allocatedWorkerResource.allocatedDiskSlots + "," + allocatedWorkerResource.allocatedMemoryMB);
+        allocatedWorkerResource.workerResource.releaseResource(
+            allocatedWorkerResource.allocatedDiskSlots, allocatedWorkerResource.allocatedMemoryMB);
+      } else {
+        LOG.warn("No AllocatedWorkerResource data for [" + ebId + "," + containerId + "]");
+        return;
       }
     }
 
@@ -411,7 +629,7 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
         return;
       } else {
         queryMasterWorkerResource = queryMasterMap.remove(queryId);
-        queryMasterWorkerResource.releaseQueryMasterTask();
+        queryMasterWorkerResource.releaseQueryMasterTask(queryMasterDefaultDiskSlot, queryMasterDefaultMemoryMB);
       }
     }
 
@@ -462,7 +680,7 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
         workerResource.setLastHeartbeat(System.currentTimeMillis());
         workerResource.setWorkerStatus(WorkerStatus.LIVE);
         if(request.getServerStatus() != null) {
-          workerResource.setMemoryMBSlots(request.getServerStatus().getSystem().getTotalMemoryMB());
+          workerResource.setMemoryMB(request.getServerStatus().getMemoryResourceMB());
           workerResource.setCpuCoreSlots(request.getServerStatus().getSystem().getAvailableProcessors());
           workerResource.setDiskSlots(request.getServerStatus().getDiskSlots());
           workerResource.setNumRunningTasks(request.getServerStatus().getRunningTaskNum());
@@ -470,7 +688,7 @@ public class TajoWorkerResourceManager implements WorkerResourceManager {
           workerResource.setFreeHeap(request.getServerStatus().getJvmHeap().getFreeHeap());
           workerResource.setTotalHeap(request.getServerStatus().getJvmHeap().getTotalHeap());
         } else {
-          workerResource.setMemoryMBSlots(4096);
+          workerResource.setMemoryMB(4096);
           workerResource.setDiskSlots(4);
           workerResource.setCpuCoreSlots(4);
         }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
index b702063..e8c9a9e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
@@ -35,12 +35,12 @@ public class WorkerResource implements Comparable<WorkerResource> {
   private int pullServerPort;
   private int httpPort;
 
-  private int diskSlots;
+  private float diskSlots;
   private int cpuCoreSlots;
-  private int memoryMBSlots;
+  private int memoryMB;
 
-  private int usedDiskSlots;
-  private int usedMemoryMBSlots;
+  private float usedDiskSlots;
+  private int usedMemoryMB;
   private int usedCpuCoreSlots;
 
   private long maxHeap;
@@ -75,28 +75,11 @@ public class WorkerResource implements Comparable<WorkerResource> {
     this.allocatedHost = allocatedHost;
   }
 
-  public void addUsedDiskSlots(int diskSlots) {
-    usedDiskSlots += diskSlots;
-  }
-
-  public void addUsedMemoryMBSlots(int memoryMBSlots) {
-    try {
-      wlock.lock();
-      usedMemoryMBSlots += memoryMBSlots;
-    } finally {
-      wlock.unlock();
-    }
-  }
-
-  public void addUsedCpuCoreSlots(int cpuCoreSlots) {
-    usedCpuCoreSlots += cpuCoreSlots;
-  }
-
-  public int getDiskSlots() {
+  public float getDiskSlots() {
     return diskSlots;
   }
 
-  public void setDiskSlots(int diskSlots) {
+  public void setDiskSlots(float diskSlots) {
     this.diskSlots = diskSlots;
   }
 
@@ -108,36 +91,40 @@ public class WorkerResource implements Comparable<WorkerResource> {
     this.cpuCoreSlots = cpuCoreSlots;
   }
 
-  public int getMemoryMBSlots() {
+  public int getMemoryMB() {
     try {
       rlock.lock();
-      return memoryMBSlots;
+      return memoryMB;
     } finally {
       rlock.unlock();
     }
   }
 
-  public void setMemoryMBSlots(int memoryMBSlots) {
+  public void setMemoryMB(int memoryMB) {
     try {
       wlock.lock();
-      this.memoryMBSlots = memoryMBSlots;
+      this.memoryMB = memoryMB;
     } finally {
       wlock.unlock();
     }
   }
 
-  public int getAvailableDiskSlots() {
+  public float getAvailableDiskSlots() {
     return diskSlots - usedDiskSlots;
   }
 
-  public int getAvailableMemoryMBSlots() {
-    return getMemoryMBSlots() - getUsedMemoryMBSlots();
+  public int getAvailableMemoryMB() {
+    return memoryMB - usedMemoryMB;
+  }
+
+  public int getAvailableCpuCoreSlots() {
+    return cpuCoreSlots - usedCpuCoreSlots;
   }
 
   @Override
   public String toString() {
-    return "host:" + allocatedHost + ", port=" + portsToStr() + ", slots=" + memoryMBSlots + ":" + cpuCoreSlots + ":" + diskSlots +
-        ", used=" + getUsedMemoryMBSlots() + ":" + usedCpuCoreSlots + ":" + usedDiskSlots;
+    return "host:" + allocatedHost + ", port=" + portsToStr() + ", slots=m:" + memoryMB + ",d:" + diskSlots +
+        ",c:" + cpuCoreSlots + ", used=m:" + usedMemoryMB + ",d:" + usedDiskSlots + ",c:" + usedCpuCoreSlots;
   }
 
   public String portsToStr() {
@@ -148,23 +135,22 @@ public class WorkerResource implements Comparable<WorkerResource> {
     this.lastHeartbeat = heartbeatTime;
   }
 
-  public int getUsedMemoryMBSlots() {
+  public int getUsedMemoryMB() {
     try {
       rlock.lock();
-      return usedMemoryMBSlots;
+      return usedMemoryMB;
     } finally {
       rlock.unlock();
     }
   }
 
-  public void setUsedMemoryMBSlots(int usedMemoryMBSlots) {
+  public void setUsedMemoryMB(int usedMemoryMB) {
     try {
       wlock.lock();
-      this.usedMemoryMBSlots = usedMemoryMBSlots;
+      this.usedMemoryMB = usedMemoryMB;
     } finally {
       wlock.unlock();
     }
-
   }
 
   public int getUsedCpuCoreSlots() {
@@ -175,7 +161,7 @@ public class WorkerResource implements Comparable<WorkerResource> {
     this.usedCpuCoreSlots = usedCpuCoreSlots;
   }
 
-  public int getUsedDiskSlots() {
+  public float getUsedDiskSlots() {
     return usedDiskSlots;
   }
 
@@ -211,33 +197,40 @@ public class WorkerResource implements Comparable<WorkerResource> {
     this.taskRunnerMode = taskRunnerMode;
   }
 
-  public void releaseResource(WorkerResource workerResource) {
+  public void releaseResource(float diskSlots, int memoryMB) {
     try {
       wlock.lock();
-      usedMemoryMBSlots = usedMemoryMBSlots - workerResource.getMemoryMBSlots();
+      usedMemoryMB = usedMemoryMB - memoryMB;
+      usedDiskSlots -= diskSlots;
+      if(usedMemoryMB < 0) {
+        LOG.warn("Used memory can't be a minus: " + usedMemoryMB);
+        usedMemoryMB = 0;
+      }
+      if(usedDiskSlots < 0) {
+        LOG.warn("Used disk slot can't be a minus: " + usedDiskSlots);
+        usedDiskSlots = 0;
+      }
     } finally {
       wlock.unlock();
     }
-
-    if(getUsedMemoryMBSlots() < 0 || usedDiskSlots < 0 || usedCpuCoreSlots < 0) {
-      LOG.warn("Used resources can't be a minus.");
-      LOG.warn(this + " ==> " + workerResource);
-    }
   }
 
-  public int getSlots() {
-    //TODO what is slot? 512MB = 1slot?
-    return getMemoryMBSlots()/512;
-  }
+  public void allocateResource(float diskSlots, int memoryMB) {
+    try {
+      wlock.lock();
+      usedMemoryMB += memoryMB;
+      usedDiskSlots += diskSlots;
 
-  public int getAvaliableSlots() {
-    //TODO what is slot? 512MB = 1slot?
-    return getAvailableMemoryMBSlots()/512;
-  }
+      if(usedMemoryMB > this.memoryMB) {
+        usedMemoryMB = this.memoryMB;
+      }
 
-  public int getUsedSlots() {
-    //TODO what is slot? 512MB = 1slot?
-    return getUsedMemoryMBSlots()/512;
+      if(usedDiskSlots > this.diskSlots) {
+        usedDiskSlots = this.diskSlots;
+      }
+    } finally {
+      wlock.unlock();
+    }
   }
 
   public int getPeerRpcPort() {
@@ -316,16 +309,14 @@ public class WorkerResource implements Comparable<WorkerResource> {
     return numQueryMasterTasks.get();
   }
 
-  public void setNumQueryMasterTasks(int numQueryMasterTasks) {
-    this.numQueryMasterTasks.set(numQueryMasterTasks);
-  }
-
-  public void addNumQueryMasterTask() {
+  public void addNumQueryMasterTask(float diskSlots, int memoryMB) {
     numQueryMasterTasks.getAndIncrement();
+    allocateResource(diskSlots, memoryMB);
   }
 
-  public void releaseQueryMasterTask() {
+  public void releaseQueryMasterTask(float diskSlots, int memoryMB) {
     numQueryMasterTasks.getAndDecrement();
+    releaseResource(diskSlots, memoryMB);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
index 2e66f98..1ce2c9f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
@@ -20,6 +20,8 @@ package org.apache.tajo.master.rm;
 
 import com.google.protobuf.RpcCallback;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.proto.YarnProtos.*;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.master.querymaster.QueryInProgress;
@@ -56,13 +58,13 @@ public interface WorkerResourceManager {
 
   public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request);
 
-  public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource);
+  public void releaseWorkerResource(ExecutionBlockId ebId, ContainerIdProto containerId);
 
   public Map<String, WorkerResource> getWorkers();
 
   public void stop();
 
-  public int getNumClusterSlots();
+  public TajoMasterProtocol.ClusterResourceSummary getClusterResourceSummary();
 
   Collection<String> getQueryMasters();
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
index 802e5ed..8b72cf9 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
@@ -36,10 +36,13 @@ import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoProtos;
+import org.apache.tajo.exception.UnimplementedException;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.YarnContainerProxy;
@@ -83,33 +86,38 @@ public class YarnTajoResourceManager implements WorkerResourceManager {
     return new ArrayList<String>();
   }
 
-  public int getNumClusterSlots() {
-    return 0;
+  public TajoMasterProtocol.ClusterResourceSummary getClusterResourceSummary() {
+    return TajoMasterProtocol.ClusterResourceSummary.newBuilder()
+        .setNumWorkers(0)
+        .setTotalCpuCoreSlots(0)
+        .setTotalDiskSlots(0)
+        .setTotalMemoryMB(0)
+        .setTotalAvailableCpuCoreSlots(0)
+        .setTotalAvailableDiskSlots(0)
+        .setTotalAvailableMemoryMB(0)
+        .build();
   }
 
   @Override
   public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request) {
-    //nothing to do
-    //yarn manages worker membership.
+    throw new UnimplementedException("workerHeartbeat");
   }
 
   @Override
-  public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource) {
-    //nothing to do
+  public void releaseWorkerResource(ExecutionBlockId ebId, YarnProtos.ContainerIdProto containerId) {
+    throw new UnimplementedException("releaseWorkerResource");
   }
 
   @Override
   public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress) {
-    //nothing to do
-    //allocateAndLaunchQueryMaster in startQueryMaster()
-    return null;
+    throw new UnimplementedException("allocateQueryMaster");
   }
 
   @Override
   public void allocateWorkerResources(
       TajoMasterProtocol.WorkerResourceAllocationRequest request,
       RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> rpcCallBack) {
-    //nothing to do
+    throw new UnimplementedException("allocateWorkerResources");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
index 5fe1b74..cf68145 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
@@ -26,6 +26,9 @@ import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 
 public class ApplicationIdUtils {
+  public static ApplicationAttemptId createApplicationAttemptId(QueryId queryId, int attemptId) {
+    return BuilderUtils.newApplicationAttemptId(queryIdToAppId(queryId), attemptId);
+  }
 
   public static ApplicationAttemptId createApplicationAttemptId(QueryId queryId) {
     return BuilderUtils.newApplicationAttemptId(queryIdToAppId(queryId), 1);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
index f0c70cf..8b9219c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
@@ -24,5 +24,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos;
 public interface ResourceAllocator {
   public void allocateTaskWorker();
   public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId);
-  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext, int numTasks);
+  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext,
+                                           int numTasks, int memoryMBPerTask);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
index 275660a..ca6ea41 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
@@ -25,7 +25,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -93,8 +95,12 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
   }
 
   @Override
-  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext, int numTasks) {
-    int clusterSlots = workerContext.getNumClusterSlots();
+  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext,
+                                           int numTasks,
+                                           int memoryMBPerTask) {
+    //TODO consider disk slot
+    TajoMasterProtocol.ClusterResourceSummary clusterResource = workerContext.getClusterResource();
+    int clusterSlots = clusterResource == null ? 0 : clusterResource.getTotalMemoryMB()/memoryMBPerTask;
     return clusterSlots == 0 ? 1: Math.min(numTasks, clusterSlots);
   }
 
@@ -226,13 +232,18 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
       CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack =
           new CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse>();
 
-      int requiredMemoryMBSlot = 512;  //TODO
-      int requiredDiskSlots = 1;  //TODO
+      //TODO consider task's resource usage pattern
+      int requiredMemoryMB = tajoConf.getIntVar(TajoConf.ConfVars.TASK_DEFAULT_MEMORY);
+      float requiredDiskSlots = tajoConf.getFloatVar(TajoConf.ConfVars.TASK_DEFAULT_DISK);
+
       TajoMasterProtocol.WorkerResourceAllocationRequest request =
           TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
-              .setMemoryMBSlots(requiredMemoryMBSlot)
-              .setDiskSlots(requiredDiskSlots)
-              .setNumWorks(event.getRequiredNum())
+              .setMinMemoryMBPerContainer(requiredMemoryMB)
+              .setMaxMemoryMBPerContainer(requiredMemoryMB)
+              .setNumContainers(event.getRequiredNum())
+              .setResourceRequestPriority(TajoMasterProtocol.ResourceRequestPriority.MEMORY)
+              .setMinDiskSlotPerContainer(requiredDiskSlots)
+              .setMaxDiskSlotPerContainer(requiredDiskSlots)
               .setExecutionBlockId(event.getExecutionBlockId().getProto())
               .build();
 
@@ -266,25 +277,26 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
           continue;
         }
       }
-      int numAllocatedWorkers = 0;
+      int numAllocatedContainers = 0;
 
       if(response != null) {
-        List<TajoMasterProtocol.WorkerAllocatedResource> workerHosts = response.getWorkerAllocatedResourceList();
+        List<TajoMasterProtocol.WorkerAllocatedResource> allocatedResources = response.getWorkerAllocatedResourceList();
         ExecutionBlockId executionBlockId = event.getExecutionBlockId();
 
         List<Container> containers = new ArrayList<Container>();
-        for(TajoMasterProtocol.WorkerAllocatedResource eachWorker: workerHosts) {
+        for(TajoMasterProtocol.WorkerAllocatedResource eachAllocatedResource: allocatedResources) {
           TajoWorkerContainer container = new TajoWorkerContainer();
           NodeIdPBImpl nodeId = new NodeIdPBImpl();
 
-          nodeId.setHost(eachWorker.getWorkerHost());
-          nodeId.setPort(eachWorker.getPeerRpcPort());
+          nodeId.setHost(eachAllocatedResource.getWorkerHost());
+          nodeId.setPort(eachAllocatedResource.getPeerRpcPort());
 
           TajoWorkerContainerId containerId = new TajoWorkerContainerId();
 
           containerId.setApplicationAttemptId(
-              ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId()));
-          containerId.setId(containerIdSeq.incrementAndGet());
+              ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId(),
+                  eachAllocatedResource.getContainerId().getAppAttemptId().getAttemptId()));
+          containerId.setId(eachAllocatedResource.getContainerId().getId());
 
           container.setId(containerId);
           container.setNodeId(nodeId);
@@ -292,10 +304,10 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
           WorkerResource workerResource = new WorkerResource();
           workerResource.setAllocatedHost(nodeId.getHost());
           workerResource.setPeerRpcPort(nodeId.getPort());
-          workerResource.setQueryMasterPort(eachWorker.getQueryMasterPort());
-          workerResource.setPullServerPort(eachWorker.getWorkerPullServerPort());
-          workerResource.setMemoryMBSlots(requiredMemoryMBSlot);
-          workerResource.setDiskSlots(requiredDiskSlots);
+          workerResource.setQueryMasterPort(eachAllocatedResource.getQueryMasterPort());
+          workerResource.setPullServerPort(eachAllocatedResource.getWorkerPullServerPort());
+          workerResource.setMemoryMB(eachAllocatedResource.getAllocatedMemoryMB());
+          workerResource.setDiskSlots(eachAllocatedResource.getAllocatedDiskSlots());
 
           container.setWorkerResource(workerResource);
 
@@ -304,32 +316,32 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
 
         SubQueryState state = queryTaskContext.getSubQuery(executionBlockId).getState();
         if (!SubQuery.isRunningState(state)) {
-          List<WorkerResource> workerResources = new ArrayList<WorkerResource>();
-          for(Container eachContainer: containers) {
-            workerResources.add(((TajoWorkerContainer)eachContainer).getWorkerResource());
-          }
           try {
-            TajoContainerProxy.releaseWorkerResource(queryTaskContext, executionBlockId, workerResources);
+            List<ContainerId> containerIds = new ArrayList<ContainerId>();
+            for(Container eachContainer: containers) {
+              containerIds.add(eachContainer.getId());
+            }
+            TajoContainerProxy.releaseWorkerResource(queryTaskContext, executionBlockId, containerIds);
           } catch (Exception e) {
             LOG.error(e.getMessage(), e);
           }
           return;
         }
 
-        if (workerHosts.size() > 0) {
+        if (allocatedResources.size() > 0) {
           if(LOG.isDebugEnabled()) {
             LOG.debug("SubQueryContainerAllocationEvent fire:" + executionBlockId);
           }
           queryTaskContext.getEventHandler().handle(new SubQueryContainerAllocationEvent(executionBlockId, containers));
         }
-        numAllocatedWorkers += workerHosts.size();
+        numAllocatedContainers += allocatedResources.size();
 
       }
-      if(event.getRequiredNum() > numAllocatedWorkers) {
+      if(event.getRequiredNum() > numAllocatedContainers) {
         ContainerAllocationEvent shortRequestEvent = new ContainerAllocationEvent(
             event.getType(), event.getExecutionBlockId(), event.getPriority(),
             event.getResource(),
-            event.getRequiredNum() - numAllocatedWorkers,
+            event.getRequiredNum() - numAllocatedContainers,
             event.isLeafQuery(), event.getProgress()
         );
         queryTaskContext.getEventHandler().handle(shortRequestEvent);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
index 4d46a45..57d99c4 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -41,6 +41,9 @@ import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.RpcConnectionPool;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.storage.v2.DiskDeviceInfo;
+import org.apache.tajo.storage.v2.DiskMountInfo;
+import org.apache.tajo.storage.v2.DiskUtil;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.util.TajoIdUtils;
@@ -103,7 +106,7 @@ public class TajoWorker extends CompositeService {
 
   private AtomicInteger numClusterNodes = new AtomicInteger();
 
-  private AtomicInteger numClusterSlots = new AtomicInteger();
+  private TajoMasterProtocol.ClusterResourceSummary clusterResource;
 
   private int httpPort;
 
@@ -350,12 +353,16 @@ public class TajoWorker extends CompositeService {
       return TajoWorker.this.numClusterNodes.get();
     }
 
-    public void setNumClusterSlots(int numClusterSlots) {
-      TajoWorker.this.numClusterSlots.set(numClusterSlots);
+    public void setClusterResource(TajoMasterProtocol.ClusterResourceSummary clusterResource) {
+      synchronized(numClusterNodes) {
+        TajoWorker.this.clusterResource = clusterResource;
+      }
     }
 
-    public int getNumClusterSlots() {
-      return TajoWorker.this.numClusterSlots.get();
+    public TajoMasterProtocol.ClusterResourceSummary getClusterResource() {
+      synchronized(numClusterNodes) {
+        return TajoWorker.this.clusterResource;
+      }
     }
 
     public InetSocketAddress getTajoMasterAddress() {
@@ -391,17 +398,18 @@ public class TajoWorker extends CompositeService {
     TajoMasterProtocol.ServerStatusProto.System systemInfo;
     List<TajoMasterProtocol.ServerStatusProto.Disk> diskInfos =
         new ArrayList<TajoMasterProtocol.ServerStatusProto.Disk>();
-    int workerDisksNum;
-    List<File> mountPaths;
+    float workerDiskSlots;
+    int workerMemoryMB;
+    List<DiskDeviceInfo> diskDeviceInfos;
 
     public WorkerHeartbeatThread() {
-      int workerMemoryMB;
       int workerCpuCoreNum;
 
       boolean dedicatedResource = systemConf.getBoolVar(ConfVars.WORKER_RESOURCE_DEDICATED);
-      
+      int workerCpuCoreSlots = Runtime.getRuntime().availableProcessors();
+
       try {
-        mountPaths = getMountPath();
+        diskDeviceInfos = DiskUtil.getDiskDeviceInfos();
       } catch (Exception e) {
         LOG.error(e.getMessage(), e);
       }
@@ -411,24 +419,23 @@ public class TajoWorker extends CompositeService {
         int totalMemory = getTotalMemoryMB();
         workerMemoryMB = (int) ((float) (totalMemory) * dedicatedMemoryRatio);
         workerCpuCoreNum = Runtime.getRuntime().availableProcessors();
-        if(mountPaths == null) {
-          workerDisksNum = ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.defaultIntVal;
+
+        if(diskDeviceInfos == null) {
+          workerDiskSlots = ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.defaultIntVal;
         } else {
-          workerDisksNum = mountPaths.size();
+          workerDiskSlots = diskDeviceInfos.size();
         }
       } else {
-        // TODO - it's a hack and it must be fixed
-        //workerMemoryMB = systemConf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB);
-        workerMemoryMB = 512 * systemConf.getIntVar(ConfVars.WORKER_EXECUTION_MAX_SLOTS);
-        workerDisksNum = systemConf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS);
+        workerMemoryMB = systemConf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB);
         workerCpuCoreNum = systemConf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES);
+        workerDiskSlots = systemConf.getFloatVar(ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS);
       }
 
       systemInfo = TajoMasterProtocol.ServerStatusProto.System.newBuilder()
           .setAvailableProcessors(workerCpuCoreNum)
           .setFreeMemoryMB(0)
           .setMaxMemoryMB(0)
-          .setTotalMemoryMB(workerMemoryMB)
+          .setTotalMemoryMB(getTotalMemoryMB())
           .build();
     }
 
@@ -472,16 +479,8 @@ public class TajoWorker extends CompositeService {
       }
 
       while(true) {
-        if(sendDiskInfoCount == 0 && mountPaths != null) {
-          for(File eachFile: mountPaths) {
-            diskInfos.clear();
-            diskInfos.add(TajoMasterProtocol.ServerStatusProto.Disk.newBuilder()
-                .setAbsolutePath(eachFile.getAbsolutePath())
-                .setTotalSpace(eachFile.getTotalSpace())
-                .setFreeSpace(eachFile.getFreeSpace())
-                .setUsableSpace(eachFile.getUsableSpace())
-                .build());
-          }
+        if(sendDiskInfoCount == 0 && diskDeviceInfos != null) {
+          getDiskUsageInfos();
         }
         TajoMasterProtocol.ServerStatusProto.JvmHeap jvmHeap =
           TajoMasterProtocol.ServerStatusProto.JvmHeap.newBuilder()
@@ -494,7 +493,8 @@ public class TajoWorker extends CompositeService {
             .addAllDisk(diskInfos)
             .setRunningTaskNum(taskRunnerManager == null ? 1 : taskRunnerManager.getNumTasks())   //TODO
             .setSystem(systemInfo)
-            .setDiskSlots(workerDisksNum)
+            .setDiskSlots(workerDiskSlots)
+            .setMemoryResourceMB(workerMemoryMB)
             .setJvmHeap(jvmHeap)
             .setQueryMasterMode(PrimitiveProtos.BoolProto.newBuilder().setValue(queryMasterMode))
             .setTaskRunnerMode(PrimitiveProtos.BoolProto.newBuilder().setValue(taskRunnerMode))
@@ -521,13 +521,11 @@ public class TajoWorker extends CompositeService {
 
           TajoMasterProtocol.TajoHeartbeatResponse response = callBack.get(2, TimeUnit.SECONDS);
           if(response != null) {
-            if(response.getNumClusterNodes() > 0) {
-              workerContext.setNumClusterNodes(response.getNumClusterNodes());
-            }
-
-            if(response.getNumClusterSlots() > 0) {
-              workerContext.setNumClusterSlots(response.getNumClusterSlots());
+            TajoMasterProtocol.ClusterResourceSummary clusterResourceSummary = response.getClusterResourceSummary();
+            if(clusterResourceSummary.getNumWorkers() > 0) {
+              workerContext.setNumClusterNodes(clusterResourceSummary.getNumWorkers());
             }
+            workerContext.setClusterResource(clusterResourceSummary);
           } else {
             if(callBack.getController().failed()) {
               throw new ServiceException(callBack.getController().errorText());
@@ -557,6 +555,24 @@ public class TajoWorker extends CompositeService {
 
       LOG.info("Worker Resource Heartbeat Thread stopped.");
     }
+
+    private void getDiskUsageInfos() {
+      diskInfos.clear();
+      for(DiskDeviceInfo eachDevice: diskDeviceInfos) {
+        List<DiskMountInfo> mountInfos = eachDevice.getMountInfos();
+        if(mountInfos != null) {
+          for(DiskMountInfo eachMount: mountInfos) {
+            File eachFile = new File(eachMount.getMountPath());
+            diskInfos.add(TajoMasterProtocol.ServerStatusProto.Disk.newBuilder()
+                .setAbsolutePath(eachFile.getAbsolutePath())
+                .setTotalSpace(eachFile.getTotalSpace())
+                .setFreeSpace(eachFile.getFreeSpace())
+                .setUsableSpace(eachFile.getUsableSpace())
+                .build());
+          }
+        }
+      }
+    }
   }
 
   private class ShutdownHook implements Runnable {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
index f931615..70a998b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.worker;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.lang.exception.ExceptionUtils;
@@ -39,7 +38,10 @@ import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.engine.planner.PlannerUtil;
-import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.engine.planner.logical.SortNode;
 import org.apache.tajo.engine.planner.physical.PhysicalExec;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.engine.query.QueryUnitRequest;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
index d694b7c..e74a09d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
@@ -64,7 +64,9 @@ public class YarnResourceAllocator extends AbstractResourceAllocator {
   }
 
   @Override
-  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext, int numTasks) {
+  public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext,
+                                           int numTasks,
+                                           int memoryMBPerTask) {
     int numClusterNodes = workerContext.getNumClusterNodes();
 
     TajoConf conf =  (TajoConf)workerContext.getQueryMaster().getConfig();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
index f9b15a7..dca200e 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
@@ -50,12 +50,13 @@ message ServerStatusProto {
     }
 
     required System system = 1;
-    required int32 diskSlots = 2;
-    repeated Disk disk = 3;
-    required int32 runningTaskNum = 4;
-    required JvmHeap jvmHeap = 5;
-    required BoolProto queryMasterMode = 6;
-    required BoolProto taskRunnerMode = 7;
+    required float diskSlots = 2;
+    required int32 memoryResourceMB = 3;
+    repeated Disk disk = 4;
+    required int32 runningTaskNum = 5;
+    required JvmHeap jvmHeap = 6;
+    required BoolProto queryMasterMode = 7;
+    required BoolProto taskRunnerMode = 8;
 }
 
 message TajoHeartbeat {
@@ -79,16 +80,37 @@ message TajoHeartbeatResponse {
       repeated string params = 2;
   }
   required BoolProto heartbeatResult = 1;
-  required int32 numClusterNodes = 2;
-  required int32 numClusterSlots = 3;
-  optional ResponseCommand responseCommand = 4;
+  required ClusterResourceSummary clusterResourceSummary = 2;
+  optional ResponseCommand responseCommand = 3;
+}
+
+message ClusterResourceSummary {
+  required int32 numWorkers = 1;
+  required int32 totalDiskSlots = 2;
+  required int32 totalCpuCoreSlots = 3;
+  required int32 totalMemoryMB = 4;
+
+  required int32 totalAvailableDiskSlots = 5;
+  required int32 totalAvailableCpuCoreSlots = 6;
+  required int32 totalAvailableMemoryMB = 7;
+}
+
+enum ResourceRequestPriority {
+    MEMORY = 1;
+    DISK = 2;
 }
 
 message WorkerResourceAllocationRequest {
     required ExecutionBlockIdProto executionBlockId = 1;
-    required int32 numWorks = 2;
-    required int32 memoryMBSlots = 3 ;
-    required int32 diskSlots = 4;
+    required ResourceRequestPriority resourceRequestPriority = 2;
+
+    required int32 numContainers = 3;
+
+    required int32 maxMemoryMBPerContainer = 4;
+    required int32 minMemoryMBPerContainer = 5;
+
+    required float maxDiskSlotPerContainer = 6;
+    required float minDiskSlotPerContainer = 7;
 }
 
 message WorkerResourceProto {
@@ -96,19 +118,25 @@ message WorkerResourceProto {
     required int32 peerRpcPort = 2;
     required int32 queryMasterPort = 3;
     required ExecutionBlockIdProto executionBlockId = 4;
-    required int32 memoryMBSlots = 5 ;
+    required int32 memoryMB = 5 ;
     required int32 diskSlots = 6;
 }
 
 message WorkerResourceReleaseRequest {
-    repeated WorkerResourceProto workerResources = 1;
+    required ExecutionBlockIdProto executionBlockId = 1;
+    repeated ContainerIdProto containerIds = 2;
 }
 
 message WorkerAllocatedResource {
-    required string workerHost = 1;
-    required int32 peerRpcPort = 2;
-    required int32 queryMasterPort = 3;
-    required int32 workerPullServerPort = 4;
+    required ContainerIdProto containerId = 1;
+    required string nodeId = 2;
+    required string workerHost = 3;
+    required int32 peerRpcPort = 4;
+    required int32 queryMasterPort = 5;
+    required int32 workerPullServerPort = 6;
+
+    required int32 allocatedMemoryMB = 7;
+    required float allocatedDiskSlots = 8;
 }
 
 message WorkerResourceAllocationResponse {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/cluster.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/cluster.jsp b/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/cluster.jsp
index d0bf887..0600145 100644
--- a/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/cluster.jsp
+++ b/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/cluster.jsp
@@ -31,10 +31,6 @@
   List<String> wokerKeys = new ArrayList<String>(workers.keySet());
   Collections.sort(wokerKeys);
 
-  int totalSlot = 0;
-  int runningSlot = 0;
-  int idleSlot = 0;
-
   int runningQueryMasterTasks = 0;
 
   Set<WorkerResource> liveWorkers = new TreeSet<WorkerResource>();
@@ -58,9 +54,6 @@
     if(eachWorker.isTaskRunnerMode()) {
       if(eachWorker.getWorkerStatus() == WorkerStatus.LIVE) {
         liveWorkers.add(eachWorker);
-        idleSlot += eachWorker.getAvaliableSlots();
-        totalSlot += eachWorker.getSlots();
-        runningSlot += eachWorker.getUsedSlots();
       } else if(eachWorker.getWorkerStatus() == WorkerStatus.DEAD) {
         deadWorkers.add(eachWorker);
       } else if(eachWorker.getWorkerStatus() == WorkerStatus.DECOMMISSION) {
@@ -94,7 +87,7 @@
   } else {
 %>
   <table width="100%" class="border_table" border="1">
-    <tr><th>No</th><th>QueryMaster</th><th>Client Port</th><th>Running QueryMaster Tasks</th><th>Heap(free/max)</th><th>Heartbeat</th><th>Status</th></tr>
+    <tr><th>No</th><th>QueryMaster</th><th>Client Port</th><th>Running Query</th><th>Heap(free/max)</th><th>Heartbeat</th><th>Status</th></tr>
 
 <%
     int no = 1;
@@ -106,7 +99,7 @@
       <td><a href='<%=queryMasterHttp%>'><%=queryMaster.getAllocatedHost() + ":" + queryMaster.getQueryMasterPort()%></a></td>
       <td width='100' align='center'><%=queryMaster.getClientPort()%></td>
       <td width='200' align='right'><%=queryMaster.getNumQueryMasterTasks()%></td>
-      <td width='200' align='left'><%=queryMaster.getFreeHeap()/1024/1024%>/<%=queryMaster.getMaxHeap()/1024/1024%> MB</td>
+      <td width='200' align='center'><%=queryMaster.getFreeHeap()/1024/1024%>/<%=queryMaster.getMaxHeap()/1024/1024%> MB</td>
       <td width='100' align='right'><%=JSPUtil.getElapsedTime(queryMaster.getLastHeartbeat(), System.currentTimeMillis())%></td>
       <td width='100' align='center'><%=queryMaster.getWorkerStatus()%></td>
     </tr>
@@ -148,7 +141,7 @@
 
   <hr/>
   <h2>Worker</h2>
-  <div>Live:<%=liveWorkers.size()%>, Dead: <%=deadWorkersHtml%>, Slot(running/idle/total): <%=runningSlot%> / <%=idleSlot%> / <%=totalSlot%></div>
+  <div>Live:<%=liveWorkers.size()%>, Dead: <%=deadWorkersHtml%></div>
   <hr/>
   <h3>Live Workers</h3>
 <%
@@ -157,7 +150,7 @@
   } else {
 %>
   <table width="100%" class="border_table" border="1">
-    <tr><th>No</th><th>Worker</th><th>PullServer Port</th><th>Running Tasks</th><th>Slot</th></th><th>Heap(free/max)</th><th>Disk</th><th>Cpu</th><th>Heartbeat</th><th>Status</th></tr>
+    <tr><th>No</th><th>Worker</th><th>PullServer<br/>Port</th><th>Running Tasks</th><th>Memory Resource<br/>(used/total)</th><th>Disk Resource<br/>(used/total)</th></th><th>Heap(free/max)</th><th>Heartbeat</th><th>Status</th></tr>
 <%
     int no = 1;
     for(WorkerResource worker: liveWorkers) {
@@ -166,12 +159,11 @@
     <tr>
       <td width='30' align='right'><%=no++%></td>
       <td><a href='<%=workerHttp%>'><%=worker.getAllocatedHost() + ":" + worker.getPeerRpcPort()%></a></td>
-      <td width='150' align='center'><%=worker.getPullServerPort()%></td>
+      <td width='80' align='center'><%=worker.getPullServerPort()%></td>
       <td width='100' align='right'><%=worker.getNumRunningTasks()%></td>
-      <td width='100' align='right'><%=worker.getUsedSlots()%>/<%=worker.getSlots()%></td>
-      <td width='100' align='left'><%=worker.getFreeHeap()/1024/1024%>/<%=worker.getMaxHeap()/1024/1024%> MB</td>
-      <td width='100' align='right'><%=worker.getUsedDiskSlots()%>/<%=worker.getDiskSlots()%></td>
-      <td width='100' align='right'><%=worker.getUsedCpuCoreSlots()%>/<%=worker.getCpuCoreSlots()%></td>
+      <td width='150' align='center'><%=worker.getUsedMemoryMB()%>/<%=worker.getMemoryMB()%></td>
+      <td width='100' align='center'><%=worker.getUsedDiskSlots()%>/<%=worker.getDiskSlots()%></td>
+      <td width='100' align='center'><%=worker.getFreeHeap()/1024/1024%>/<%=worker.getMaxHeap()/1024/1024%> MB</td>
       <td width='100' align='right'><%=JSPUtil.getElapsedTime(worker.getLastHeartbeat(), System.currentTimeMillis())%></td>
       <td width='100' align='center'><%=worker.getWorkerStatus()%></td>
     </tr>
@@ -196,7 +188,7 @@
   } else {
 %>
   <table width="100%" class="border_table" border="1">
-    <tr><th>No</th><th>Worker</th><th>PullServer Port</th><th>Slot</th></th><th>Heap(free/max)</th><th>Disk</th><th>Cpu</th><th>Status</th></tr>
+    <tr><th>No</th><th>Worker</th><th>PullServer Port</th><th>Running Tasks</th><th>Memory Resource</th><th>Disk Resource</th></th><th>Heap(free/max)</th><th>Heartbeat</th><th>Status</th></tr>
 <%
       int no = 1;
       for(WorkerResource worker: deadWorkers) {
@@ -205,10 +197,9 @@
       <td width='30' align='right'><%=no++%></td>
       <td><%=worker.getAllocatedHost() + ":" + worker.getPeerRpcPort()%></td>
       <td width='150' align='center'><%=worker.getPullServerPort()%></td>
-      <td width='100' align='right'><%=worker.getUsedSlots()%>/<%=worker.getSlots()%></td>
-      <td width='100' align='left'><%=worker.getFreeHeap()/1024/1024%>/<%=worker.getMaxHeap()/1024/1024%> MB</td>
+      <td width='100' align='right'><%=worker.getUsedMemoryMB()%>/<%=worker.getMemoryMB()%></td>
       <td width='100' align='right'><%=worker.getUsedDiskSlots()%>/<%=worker.getDiskSlots()%></td>
-      <td width='100' align='right'><%=worker.getUsedCpuCoreSlots()%>/<%=worker.getCpuCoreSlots()%></td>
+      <td width='100' align='left'><%=worker.getFreeHeap()/1024/1024%>/<%=worker.getMaxHeap()/1024/1024%> MB</td>
       <td width='100' align='center'><%=worker.getWorkerStatus()%></td>
     </tr>
 <%

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/index.jsp b/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/index.jsp
index fd5fa47..f652ea5 100644
--- a/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/index.jsp
+++ b/tajo-core/tajo-core-backend/src/main/resources/webapps/admin/index.jsp
@@ -29,6 +29,7 @@
 <%@ page import="org.apache.hadoop.util.StringUtils" %>
 <%@ page import="org.apache.hadoop.fs.FileSystem" %>
 <%@ page import="org.apache.tajo.conf.TajoConf" %>
+<%@ page import="org.apache.tajo.ipc.TajoMasterProtocol" %>
 
 <%
   TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
@@ -44,9 +45,9 @@
   int numDeadQueryMasters = 0;
   int runningQueryMasterTask = 0;
 
-  int totalSlot = 0;
-  int runningSlot = 0;
-  int idleSlot = 0;
+
+  TajoMasterProtocol.ClusterResourceSummary clusterResourceSummary =
+          master.getContext().getResourceManager().getClusterResourceSummary();
 
   for(WorkerResource eachWorker: workers.values()) {
     if(eachWorker.getWorkerStatus() == WorkerStatus.LIVE) {
@@ -58,9 +59,6 @@
       if(eachWorker.isTaskRunnerMode()) {
         numWorkers++;
         numLiveWorkers++;
-        idleSlot += eachWorker.getAvaliableSlots();
-        totalSlot += eachWorker.getSlots();
-        runningSlot += eachWorker.getUsedSlots();
       }
     } else if(eachWorker.getWorkerStatus() == WorkerStatus.DEAD) {
       if(eachWorker.isQueryMasterMode()) {
@@ -143,24 +141,24 @@
 
   <h3>Cluster Summary</h3>
   <table width="100%" class="border_table" border="1">
-    <tr><th>Type</th><th>Total</th><th>Live</th><th>Dead</th><th>Total Slots</th><th>Running Slots</th><th>Idle Slots</th></tr>
+    <tr><th>Type</th><th>Total</th><th>Live</th><th>Dead</th><th>Running Master</th><th>Memory Resource<br/>(used/total)</th><th>Disk Resource<br/>(used/total)</th></tr>
     <tr>
       <td><a href='cluster.jsp'>Query Master</a></td>
       <td align='right'><%=numQueryMasters%></td>
       <td align='right'><%=numLiveQueryMasters%></td>
       <td align='right'><%=numDeadQueryMastersHtml%></td>
-      <td align='right'>-</td>
       <td align='right'><%=runningQueryMasterTask%></td>
-      <td align='right'>-</td>
+      <td align='center'>-</td>
+      <td align='center'>-</td>
     </tr>
     <tr>
       <td><a href='cluster.jsp'>Worker</a></td>
       <td align='right'><%=numWorkers%></td>
       <td align='right'><%=numLiveWorkers%></td>
       <td align='right'><%=numDeadWorkersHtml%></td>
-      <td align='right'><%=totalSlot%></td>
-      <td align='right'><%=runningSlot%></td>
-      <td align='right'><%=idleSlot%></td>
+      <td align='right'>-</td>
+      <td align='center'><%=clusterResourceSummary.getTotalMemoryMB() - clusterResourceSummary.getTotalAvailableMemoryMB()%>/<%=clusterResourceSummary.getTotalMemoryMB()%></td>
+      <td align='center'><%=clusterResourceSummary.getTotalDiskSlots() - clusterResourceSummary.getTotalAvailableDiskSlots()%>/<%=clusterResourceSummary.getTotalDiskSlots()%></td>
     </tr>
   </table>
   <p/>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/528c914f/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
index 9ea07de..b7c8512 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -96,6 +96,9 @@ public class TajoTestingCluster {
       );
       conf.set(ConfVars.RESOURCE_MANAGER_CLASS.varname, System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname));
     }
+    conf.setInt(ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB.varname, 1024);
+    conf.setFloat(ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS.varname, 2.0f);
+
     this.standbyWorkerMode = conf.getVar(ConfVars.RESOURCE_MANAGER_CLASS)
         .indexOf(TajoWorkerResourceManager.class.getName()) >= 0;
     conf.set(CommonTestingUtil.TAJO_TEST, "TRUE");


[15/18] git commit: TAJO-284: Add table partitioning entry to Catalog. (jaehwa)

Posted by ji...@apache.org.
TAJO-284: Add table partitioning entry to Catalog. (jaehwa)


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

Branch: refs/heads/DAG-execplan
Commit: 0b0de13b2444f9e75ad3e1f42cba51ddb1f86dc2
Parents: 29a0aa0
Author: blrunner <jh...@gruter.com>
Authored: Tue Dec 3 18:29:22 2013 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Tue Dec 3 18:29:22 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../org/apache/tajo/algebra/CreateTable.java    |  11 +-
 .../apache/tajo/catalog/CatalogConstants.java   |   1 +
 .../org/apache/tajo/catalog/CatalogUtil.java    |   3 +
 .../org/apache/tajo/catalog/DDLBuilder.java     |  70 +++
 .../java/org/apache/tajo/catalog/Schema.java    |   2 +-
 .../java/org/apache/tajo/catalog/TableDesc.java |  33 +-
 .../tajo/catalog/partition/Partitions.java      | 349 +++++++++++++
 .../tajo/catalog/partition/Specifier.java       | 128 +++++
 .../src/main/proto/CatalogProtos.proto          |  20 +
 tajo-catalog/tajo-catalog-server/pom.xml        |   4 +
 .../org/apache/tajo/catalog/CatalogServer.java  |   5 +-
 .../tajo/catalog/store/AbstractDBStore.java     | 184 ++++++-
 .../apache/tajo/catalog/store/DerbyStore.java   | 488 +++++++++++++------
 .../apache/tajo/catalog/store/MySQLStore.java   | 204 +++++---
 .../org/apache/tajo/catalog/TestCatalog.java    | 225 ++++++++-
 .../org/apache/tajo/catalog/TestDBStore.java    | 174 +++++++
 .../main/java/org/apache/tajo/cli/TajoCli.java  |  36 ++
 .../apache/tajo/engine/planner/LogicalPlan.java |  13 +-
 .../tajo/engine/planner/LogicalPlanner.java     | 135 ++++-
 .../engine/planner/logical/CreateTableNode.java |  18 +-
 .../engine/planner/logical/StoreTableNode.java  |  26 +-
 .../org/apache/tajo/master/GlobalEngine.java    |   9 +-
 .../tajo/master/TajoMasterClientService.java    |   6 +-
 .../src/main/proto/ClientProtos.proto           |   1 +
 .../org/apache/tajo/client/TestTajoClient.java  | 122 +++++
 26 files changed, 2022 insertions(+), 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ab0e893..8c53d9b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -46,6 +46,8 @@ Release 0.8.0 - unreleased
 
   IMPROVEMENTS
 
+    TAJO-284: Add table partitioning entry to Catalog. (jaehwa)
+
     TAJO-317: Improve TajoResourceManager to support more elaborate resource 
     management. (Keuntae Park via jihoon)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
index 6e36f3a..41276ad 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
@@ -279,7 +279,6 @@ public class CreateTable extends Expr {
   }
 
   public static class RangePartitionSpecifier extends PartitionSpecifier {
-    String name;
     Expr end;
     boolean maxValue;
 
@@ -293,10 +292,6 @@ public class CreateTable extends Expr {
       maxValue = true;
     }
 
-    public String getName() {
-      return name;
-    }
-
     public Expr getEnd() {
       return end;
     }
@@ -320,10 +315,14 @@ public class CreateTable extends Expr {
   }
 
   public static class PartitionSpecifier {
-    String name;
+    private String name;
 
     public PartitionSpecifier(String name) {
       this.name = name;
     }
+
+    public String getName() {
+      return this.name;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java
index 4b1f794..ed23b08 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java
@@ -35,6 +35,7 @@ public class CatalogConstants {
   public static final String TB_OPTIONS = "OPTIONS";
   public static final String TB_INDEXES = "INDEXES";
   public static final String TB_STATISTICS = "STATS";
+  public static final String TB_PARTTIONS = "PARTITIONS";
   public static final String C_TABLE_ID = "TABLE_ID";
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
index 6b4848c..dc91035 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@ -27,6 +27,7 @@ import org.apache.tajo.common.TajoDataTypes.DataType;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.Statement;
+import java.sql.PreparedStatement;
 import java.sql.Wrapper;
 import java.util.Collection;
 
@@ -148,6 +149,8 @@ public class CatalogUtil {
       try{
         if(w instanceof Statement){
           ((Statement)w).close();
+        } else if(w instanceof PreparedStatement){
+            ((PreparedStatement)w).close();
         } else if(w instanceof ResultSet){
           ((ResultSet)w).close();
         } else if(w instanceof Connection){

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
index e6cc46d..a9d0f03 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
@@ -18,6 +18,9 @@
 
 package org.apache.tajo.catalog;
 
+import org.apache.tajo.catalog.partition.Partitions;
+import org.apache.tajo.catalog.partition.Specifier;
+import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
 
 import java.util.Map;
@@ -38,6 +41,10 @@ public class DDLBuilder {
     buildWithClause(sb, desc.getMeta());
     buildLocationClause(sb, desc);
 
+    if (desc.getPartitions() != null) {
+      buildPartitionClause(sb, desc);
+    }
+
     sb.append(";");
     return sb.toString();
   }
@@ -87,4 +94,67 @@ public class DDLBuilder {
   private static void buildLocationClause(StringBuilder sb, TableDesc desc) {
     sb.append(" LOCATION '").append(desc.getPath()).append("'");
   }
+
+  private static void buildPartitionClause(StringBuilder sb, TableDesc desc) {
+    Partitions partitions = desc.getPartitions();
+
+    sb.append(" PARTITION BY ");
+    sb.append(partitions.getPartitionsType().name());
+
+    // columns
+    sb.append("(");
+    int columnCount = 0;
+    for(Column column: partitions.getColumns()) {
+      for(Column targetColumn: desc.getSchema().getColumns()) {
+        if (column.getColumnName().equals(targetColumn.getColumnName()))  {
+          if (columnCount > 0)
+            sb.append(",");
+
+          sb.append(column.getColumnName());
+          columnCount++;
+        }
+      }
+    }
+    sb.append(")");
+
+    // specifier
+    if (partitions.getSpecifiers() != null
+        && !partitions.getPartitionsType().equals(CatalogProtos.PartitionsType.COLUMN)) {
+
+      sb.append(" (");
+      for(int i = 0; i < partitions.getSpecifiers().size(); i++) {
+        Specifier specifier = partitions.getSpecifiers().get(i);
+        if (i > 0)
+          sb.append(",");
+
+        sb.append(" PARTITION");
+
+        if (!specifier.getName().isEmpty())
+          sb.append(" ").append(specifier.getName());
+
+        if (partitions.getPartitionsType().equals(CatalogProtos.PartitionsType.LIST)) {
+          if (!specifier.getExpressions().isEmpty()) {
+            sb.append(" VALUES (");
+            String[] expressions = specifier.getExpressions().split("\\,");
+            for(int j = 0; j < expressions.length; j++) {
+              if (j > 0)
+                sb.append(",");
+              sb.append("'").append(expressions[j]).append("'");
+            }
+            sb.append(")");
+
+          }
+        } else if (partitions.getPartitionsType().equals(CatalogProtos.PartitionsType.RANGE))  {
+          sb.append(" VALUES LESS THAN (");
+          if (!specifier.getExpressions().isEmpty()) {
+            sb.append(specifier.getExpressions());
+          } else {
+            sb.append("MAXVALUE");
+          }
+          sb.append(")");
+        }
+      }
+      sb.append(")");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
index 7c0de81..8a2d028 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
@@ -212,7 +212,7 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
     }
   }
 
-	@Override
+  @Override
 	public boolean equals(Object o) {
 		if (o instanceof Schema) {
 		  Schema other = (Schema) o;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
index f59feef..458a99a 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
@@ -21,8 +21,11 @@ package org.apache.tajo.catalog;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.annotations.Expose;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
+import org.apache.tajo.catalog.partition.Partitions;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
 import org.apache.tajo.catalog.statistics.TableStats;
@@ -30,6 +33,8 @@ import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.json.GsonObject;
 
 public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Cloneable {
+  private final Log LOG = LogFactory.getLog(TableDesc.class);
+
   protected TableDescProto.Builder builder = null;
   
 	@Expose protected String tableName; // required
@@ -37,6 +42,7 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
   @Expose protected TableMeta meta; // required
   @Expose protected Path uri; // required
   @Expose	protected TableStats stats; // optional
+  @Expose protected Partitions partitions; //optional
   
 	public TableDesc() {
 		builder = TableDescProto.newBuilder();
@@ -48,7 +54,7 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
 	  this.tableName = tableName.toLowerCase();
     this.schema = schema;
 	  this.meta = info;
-	  this.uri = path;	   
+	  this.uri = path;
 	}
 	
 	public TableDesc(String tableName, Schema schema, StoreType type, Options options, Path path) {
@@ -58,6 +64,9 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
 	public TableDesc(TableDescProto proto) {
 	  this(proto.getId(), new Schema(proto.getSchema()), new TableMeta(proto.getMeta()), new Path(proto.getPath()));
     this.stats = new TableStats(proto.getStats());
+    if (proto.getPartitions() != null && !proto.getPartitions().toString().isEmpty()) {
+      this.partitions = new Partitions(proto.getPartitions());
+    }
 	}
 	
 	public void setName(String tableId) {
@@ -104,8 +113,20 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
   public TableStats getStats() {
     return this.stats;
   }
-	
-	public boolean equals(Object object) {
+
+  public boolean hasPartitions() {
+    return this.partitions != null;
+  }
+
+  public Partitions getPartitions() {
+    return partitions;
+  }
+
+  public void setPartitions(Partitions partitions) {
+    this.partitions = partitions;
+  }
+
+  public boolean equals(Object object) {
     if(object instanceof TableDesc) {
       TableDesc other = (TableDesc) object;
       
@@ -123,6 +144,7 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
     desc.meta = (TableMeta) meta.clone();
     desc.uri = uri;
     desc.stats = stats != null ? (TableStats) stats.clone() : null;
+    desc.partitions = partitions != null ? (Partitions) partitions.clone() : null;
 	  
 	  return desc;
 	}
@@ -154,7 +176,10 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
       builder.setPath(this.uri.toString());
     }
     if (this.stats != null) {
-      builder.setStats(stats.getProto());
+      builder.setStats(this.stats.getProto());
+    }
+    if (this.partitions != null) {
+      builder.setPartitions(this.partitions.getProto());
     }
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Partitions.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Partitions.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Partitions.java
new file mode 100644
index 0000000..c82f0cb
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Partitions.java
@@ -0,0 +1,349 @@
+/**
+ * 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.catalog.partition;
+
+import com.google.common.collect.ImmutableList;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.annotations.Expose;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.exception.AlreadyExistsFieldException;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.util.TUtil;
+
+import java.util.*;
+
+public class Partitions implements ProtoObject<CatalogProtos.PartitionsProto>, Cloneable, GsonObject {
+
+  private static final Log LOG = LogFactory.getLog(Partitions.class);
+
+  @Expose protected CatalogProtos.PartitionsType partitionsType; //required
+  @Expose protected List<Column> columns; //required
+  @Expose protected int numPartitions; //optional
+  @Expose protected List<Specifier> specifiers; //optional
+  @Expose protected Map<String, Integer> columnsByQialifiedName = null;
+  @Expose protected Map<String, List<Integer>> columnsByName = null;
+
+  private CatalogProtos.PartitionsProto.Builder builder = CatalogProtos.PartitionsProto.newBuilder();
+
+  public Partitions() {
+    this.columns = new ArrayList<Column>();
+    this.columnsByQialifiedName = new TreeMap<String, Integer>();
+    this.columnsByName = new HashMap<String, List<Integer>>();
+  }
+
+  public Partitions(Partitions partition) {
+    this();
+    this.partitionsType = partition.partitionsType;
+    this.columns.addAll(partition.columns);
+    this.columnsByQialifiedName.putAll(partition.columnsByQialifiedName);
+    this.columnsByName.putAll(partition.columnsByName);
+    this.numPartitions = partition.numPartitions;
+    this.specifiers = partition.specifiers;
+  }
+
+  public Partitions(CatalogProtos.PartitionsType partitionsType, Column[] columns, int numPartitions,
+                   List<Specifier> specifiers) {
+    this();
+    this.partitionsType = partitionsType;
+    for (Column c : columns) {
+      addColumn(c);
+    }
+    this.numPartitions = numPartitions;
+    this.specifiers = specifiers;
+  }
+
+  public Partitions(CatalogProtos.PartitionsProto proto) {
+    this.partitionsType = proto.getPartitionsType();
+    this.columns = new ArrayList<Column>();
+    this.columnsByQialifiedName = new HashMap<String, Integer>();
+    this.columnsByName = new HashMap<String, List<Integer>>();
+    for (CatalogProtos.ColumnProto colProto : proto.getColumnsList()) {
+      Column tobeAdded = new Column(colProto);
+      columns.add(tobeAdded);
+      if (tobeAdded.hasQualifier()) {
+        columnsByQialifiedName.put(tobeAdded.getQualifier() + "." + tobeAdded.getColumnName(),
+            columns.size() - 1);
+      } else {
+        columnsByQialifiedName.put(tobeAdded.getColumnName(), columns.size() - 1);
+      }
+      if (columnsByName.containsKey(tobeAdded.getColumnName())) {
+        columnsByName.get(tobeAdded.getColumnName()).add(columns.size() - 1);
+      } else {
+        columnsByName.put(tobeAdded.getColumnName(), TUtil.newList(columns.size() - 1));
+      }
+    }
+    this.numPartitions = proto.getNumPartitions();
+    if(proto.getSpecifiersList() != null) {
+      this.specifiers = TUtil.newList();
+      for(CatalogProtos.SpecifierProto specifier: proto.getSpecifiersList()) {
+        this.specifiers.add(new Specifier(specifier));
+      }
+    }
+  }
+
+  /**
+   * Set a qualifier to this schema.
+   * This changes the qualifier of all columns except for not-qualified columns.
+   *
+   * @param qualifier The qualifier
+   */
+  public void setQualifier(String qualifier) {
+    setQualifier(qualifier, false);
+  }
+
+  /**
+   * Set a qualifier to this schema. This changes the qualifier of all columns if force is true.
+   * Otherwise, it changes the qualifier of all columns except for non-qualified columns
+   *
+   * @param qualifier The qualifier
+   * @param force     If true, all columns' qualifiers will be changed. Otherwise,
+   *                  only qualified columns' qualifiers will
+   *                  be changed.
+   */
+  public void setQualifier(String qualifier, boolean force) {
+    columnsByQialifiedName.clear();
+
+    for (int i = 0; i < getColumnNum(); i++) {
+      if (!force && columns.get(i).hasQualifier()) {
+        continue;
+      }
+      columns.get(i).setQualifier(qualifier);
+      columnsByQialifiedName.put(columns.get(i).getQualifiedName(), i);
+    }
+  }
+
+  public int getColumnNum() {
+    return this.columns.size();
+  }
+
+  public Column getColumn(int id) {
+    return columns.get(id);
+  }
+
+  public Column getColumnByFQN(String qualifiedName) {
+    Integer cid = columnsByQialifiedName.get(qualifiedName.toLowerCase());
+    return cid != null ? columns.get(cid) : null;
+  }
+
+  public Column getColumnByName(String colName) {
+    String normalized = colName.toLowerCase();
+    List<Integer> list = columnsByName.get(normalized);
+
+    if (list == null || list.size() == 0) {
+      return null;
+    }
+
+    if (list.size() == 1) {
+      return columns.get(list.get(0));
+    } else {
+      StringBuilder sb = new StringBuilder();
+      boolean first = true;
+      for (Integer id : list) {
+        if (first) {
+          first = false;
+        } else {
+          sb.append(", ");
+        }
+        sb.append(columns.get(id));
+      }
+      throw new RuntimeException("Ambiguous Column Name: " + sb.toString());
+    }
+  }
+
+  public int getColumnId(String qualifiedName) {
+    return columnsByQialifiedName.get(qualifiedName.toLowerCase());
+  }
+
+  public int getColumnIdByName(String colName) {
+    for (Column col : columns) {
+      if (col.getColumnName().equals(colName.toLowerCase())) {
+        return columnsByQialifiedName.get(col.getQualifiedName());
+      }
+    }
+    return -1;
+  }
+
+  public List<Column> getColumns() {
+    return ImmutableList.copyOf(columns);
+  }
+
+  public void setColumns(List<Column> columns) {
+    this.columns = columns;
+  }
+
+  public boolean contains(String colName) {
+    return columnsByQialifiedName.containsKey(colName.toLowerCase());
+
+  }
+
+  public boolean containsAll(Collection<Column> columns) {
+    return columns.containsAll(columns);
+  }
+
+  public synchronized Partitions addColumn(String name, TajoDataTypes.Type type) {
+    if (type == TajoDataTypes.Type.CHAR) {
+      return addColumn(name, CatalogUtil.newDataTypeWithLen(type, 1));
+    }
+    return addColumn(name, CatalogUtil.newSimpleDataType(type));
+  }
+
+  public synchronized Partitions addColumn(String name, TajoDataTypes.Type type, int length) {
+    return addColumn(name, CatalogUtil.newDataTypeWithLen(type, length));
+  }
+
+  public synchronized Partitions addColumn(String name, TajoDataTypes.DataType dataType) {
+    String normalized = name.toLowerCase();
+    if (columnsByQialifiedName.containsKey(normalized)) {
+      LOG.error("Already exists column " + normalized);
+      throw new AlreadyExistsFieldException(normalized);
+    }
+
+    Column newCol = new Column(normalized, dataType);
+    columns.add(newCol);
+    columnsByQialifiedName.put(newCol.getQualifiedName(), columns.size() - 1);
+    columnsByName.put(newCol.getColumnName(), TUtil.newList(columns.size() - 1));
+
+    return this;
+  }
+
+  public synchronized void addColumn(Column column) {
+    addColumn(column.getQualifiedName(), column.getDataType());
+  }
+
+  public synchronized void addColumns(Partitions schema) {
+    for (Column column : schema.getColumns()) {
+      addColumn(column);
+    }
+  }
+
+  public synchronized void addSpecifier(Specifier specifier) {
+    if(specifiers == null)
+      specifiers = TUtil.newList();
+
+    specifiers.add(specifier);
+  }
+
+  public CatalogProtos.PartitionsType getPartitionsType() {
+    return partitionsType;
+  }
+
+  public void setPartitionsType(CatalogProtos.PartitionsType partitionsType) {
+    this.partitionsType = partitionsType;
+  }
+
+  public int getNumPartitions() {
+    return numPartitions;
+  }
+
+  public void setNumPartitions(int numPartitions) {
+    this.numPartitions = numPartitions;
+  }
+
+  public List<Specifier> getSpecifiers() {
+    return specifiers;
+  }
+
+  public void setSpecifiers(List<Specifier> specifiers) {
+    this.specifiers = specifiers;
+  }
+
+  public Map<String, Integer> getColumnsByQialifiedName() {
+    return columnsByQialifiedName;
+  }
+
+  public void setColumnsByQialifiedName(Map<String, Integer> columnsByQialifiedName) {
+    this.columnsByQialifiedName = columnsByQialifiedName;
+  }
+
+  public Map<String, List<Integer>> getColumnsByName() {
+    return columnsByName;
+  }
+
+  public void setColumnsByName(Map<String, List<Integer>> columnsByName) {
+    this.columnsByName = columnsByName;
+  }
+
+  public boolean equals(Object o) {
+    if (o instanceof Partitions) {
+      Partitions other = (Partitions) o;
+      return getProto().equals(other.getProto());
+    }
+    return false;
+  }
+
+  public Object clone() throws CloneNotSupportedException {
+    Partitions clone = (Partitions) super.clone();
+    clone.builder = CatalogProtos.PartitionsProto.newBuilder();
+    clone.setPartitionsType(this.partitionsType);
+    clone.setColumns(this.columns);
+    clone.setNumPartitions(this.numPartitions);
+    clone.specifiers = new ArrayList<Specifier>(this.specifiers);
+
+    return clone;
+  }
+
+  @Override
+  public CatalogProtos.PartitionsProto getProto() {
+    if (builder == null) {
+      builder = CatalogProtos.PartitionsProto.newBuilder();
+    }
+    if (this.partitionsType != null) {
+      builder.setPartitionsType(this.partitionsType);
+    }
+    builder.clearColumns();
+    if (this.columns != null) {
+      for (Column col : columns) {
+        builder.addColumns(col.getProto());
+      }
+    }
+    builder.setNumPartitions(numPartitions);
+
+    if (this.specifiers != null) {
+      for(Specifier specifier: specifiers) {
+        builder.addSpecifiers(specifier.getProto());
+      }
+    }
+    return builder.build();
+  }
+
+  public String toString() {
+    Gson gson = new GsonBuilder().setPrettyPrinting().
+        excludeFieldsWithoutExposeAnnotation().create();
+    return gson.toJson(this);
+  }
+
+  @Override
+  public String toJson() {
+    return CatalogGsonHelper.toJson(this, Partitions.class);
+
+  }
+
+  public Column[] toArray() {
+    return this.columns.toArray(new Column[this.columns.size()]);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Specifier.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Specifier.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Specifier.java
new file mode 100644
index 0000000..feb8a33
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/Specifier.java
@@ -0,0 +1,128 @@
+/**
+ * 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.catalog.partition;
+
+import com.google.common.base.Objects;
+import com.google.gson.Gson;
+import com.google.gson.annotations.Expose;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.util.TUtil;
+
+public class Specifier implements ProtoObject<CatalogProtos.SpecifierProto>, Cloneable,
+    GsonObject {
+
+  private static final Log LOG = LogFactory.getLog(Specifier.class);
+  protected CatalogProtos.SpecifierProto.Builder builder = null;
+
+
+  @Expose protected String name;
+  @Expose protected String expressions;
+
+  public Specifier() {
+    builder = CatalogProtos.SpecifierProto.newBuilder();
+  }
+
+  public Specifier(String name) {
+    this();
+    this.name = name;
+  }
+
+  public Specifier(String name, String expressions) {
+    this();
+    this.name = name;
+    this.expressions = expressions;
+  }
+
+  public Specifier(CatalogProtos.SpecifierProto proto) {
+    this();
+    this.name = proto.getName().toLowerCase();
+    this.expressions = proto.getExpressions();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getExpressions() {
+    return expressions;
+  }
+
+  public void setExpressions(String expressions) {
+    this.expressions = expressions;
+  }
+
+  public boolean equals(Object o) {
+    if (o instanceof Specifier) {
+      Specifier other = (Specifier)o;
+      boolean eq = TUtil.checkEquals(this.name, other.name);
+      eq = eq && TUtil.checkEquals(this.expressions, other.expressions);
+      return  eq;
+    }
+    return false;
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(this.name, this.expressions);
+
+  }
+
+  public Object clone() throws CloneNotSupportedException {
+    Specifier clone = (Specifier) super.clone();
+    clone.builder = CatalogProtos.SpecifierProto.newBuilder();
+    clone.name = this.name;
+    clone.expressions = this.expressions;
+    return clone;
+  }
+
+  public String toString() {
+    Gson gson = CatalogGsonHelper.getPrettyInstance();
+    return gson.toJson(this);
+  }
+
+  @Override
+  public CatalogProtos.SpecifierProto getProto() {
+    if(builder == null) {
+      builder = CatalogProtos.SpecifierProto.newBuilder();
+    }
+
+    if(this.name != null) {
+      builder.setName(this.name);
+    }
+
+    if(this.expressions != null) {
+      builder.setExpressions(this.expressions);
+    }
+
+    return builder.build();
+  }
+
+  @Override
+  public String toJson() {
+    return CatalogGsonHelper.toJson(this, Specifier.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/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 acfa4fb..e5af491 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -50,6 +50,13 @@ enum CompressType {
     LZ = 6;
 }
 
+enum PartitionsType {
+    RANGE = 0;
+    HASH = 1;
+    LIST = 2;
+    COLUMN = 3;
+}
+
 message ColumnMetaProto {
     required DataType dataType = 1;
     required bool compressed = 2;
@@ -106,6 +113,7 @@ message TableDescProto {
 	required TableProto meta = 3;
 	required SchemaProto schema = 4;
 	optional TableStatsProto stats = 5;
+	optional PartitionsProto partitions = 6;
 }
 
 enum FunctionType {
@@ -225,3 +233,15 @@ message SortSpecProto {
   optional bool ascending = 2 [default = true];
   optional bool nullFirst = 3 [default = false];
 }
+
+message PartitionsProto {
+  required PartitionsType partitionsType = 1;
+  repeated ColumnProto columns = 2;
+  optional int32 numPartitions = 3;
+  repeated SpecifierProto specifiers = 4;
+}
+
+message SpecifierProto {
+	optional string name = 1;
+	optional string expressions = 2;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-server/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/pom.xml b/tajo-catalog/tajo-catalog-server/pom.xml
index ca8a3ef..e1105eb 100644
--- a/tajo-catalog/tajo-catalog-server/pom.xml
+++ b/tajo-catalog/tajo-catalog-server/pom.xml
@@ -127,6 +127,10 @@
       <groupId>org.apache.tajo</groupId>
       <artifactId>tajo-rpc</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-algebra</artifactId>
+    </dependency>
 
     <dependency>
       <groupId>com.google.protobuf</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/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 dc279bc..e6566af 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
@@ -253,9 +253,12 @@ public class CatalogServer extends AbstractService {
         descBuilder.setMeta(tableDesc.getMeta());
         descBuilder.setSchema(tableDesc.getSchema());
 
+        if( tableDesc.getPartitions() != null
+            && !tableDesc.getPartitions().toString().isEmpty()) {
+          descBuilder.setPartitions(tableDesc.getPartitions());
+        }
 
         store.addTable(new TableDesc(descBuilder.build()));
-
       } catch (IOException ioe) {
         LOG.error(ioe.getMessage(), ioe);
         return BOOL_FALSE;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/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 6a76794..3414e83 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
@@ -26,6 +26,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.partition.Partitions;
+import org.apache.tajo.catalog.partition.Specifier;
+import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
@@ -37,6 +40,8 @@ import org.apache.tajo.exception.InternalException;
 import java.io.IOException;
 import java.sql.*;
 import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map.Entry;
 
@@ -47,6 +52,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   protected String connectionPassword;
   protected String catalogUri;
   private Connection conn;
+  protected Map<String, Boolean> baseTableMaps = new HashMap<String, Boolean>();
 
   protected static final int VERSION = 1;
 
@@ -202,6 +208,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   @Override
   public void addTable(final TableDesc table) throws IOException {
     Statement stmt = null;
+    PreparedStatement pstmt = null;
+
     ResultSet res;
 
     String sql =
@@ -245,7 +253,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
 
       String optSql = String.format("INSERT INTO %s (%s, key_, value_) VALUES(?, ?, ?)", TB_OPTIONS, C_TABLE_ID);
-      PreparedStatement pstmt = getConnection().prepareStatement(optSql);
+      pstmt = getConnection().prepareStatement(optSql);
       try {
         for (Entry<String, String> entry : table.getMeta().toMap().entrySet()) {
           pstmt.setString(1, table.getName());
@@ -272,9 +280,94 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         stmt.addBatch(sql);
         stmt.executeBatch();
       }
+
+      //Partition
+      if (table.getPartitions() != null && !table.getPartitions().toString().isEmpty()) {
+        try {
+          Partitions partitions = table.getPartitions();
+          List<Column> columnList = partitions.getColumns();
+
+          // Find columns which used for a partitioned table.
+          StringBuffer columns = new StringBuffer();
+          for(Column eachColumn : columnList) {
+            sql = "SELECT column_id from " + TB_COLUMNS + " WHERE TID "
+                + " = " + tid + " AND column_name = '" + eachColumn.getColumnName() + "'";
+
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(sql);
+            }
+            res = stmt.executeQuery(sql);
+            if (!res.next()) {
+              throw new IOException("ERROR: there is no columnId matched to "
+                  + table.getName());
+            }
+            columnId = res.getInt("column_id");
+
+            if (columns.length() > 0) {
+              columns.append(",");
+            }
+            columns.append(columnId);
+          }
+
+          // Set default partition name. But if user named to subpartition, it would be updated.
+//          String partitionName = partitions.getPartitionsType().name() + "_" + table.getName();
+
+          sql = "INSERT INTO " + TB_PARTTIONS + " (name, TID, "
+              + " type, quantity, columns, expressions) VALUES (?, ?, ?, ?, ?, ?) ";
+          pstmt = getConnection().prepareStatement(sql);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(sql);
+          }
+
+          // Find information for subpartitions
+          if (partitions.getSpecifiers() != null) {
+            int count = 1;
+            if (partitions.getSpecifiers().size() == 0) {
+              pstmt.clearParameters();
+              pstmt.setString(1, null);
+              pstmt.setInt(2, tid);
+              pstmt.setString(3, partitions.getPartitionsType().name());
+              pstmt.setInt(4, partitions.getNumPartitions());
+              pstmt.setString(5, columns.toString());
+              pstmt.setString(6, null);
+              pstmt.addBatch();
+            } else {
+              for(Specifier specifier: partitions.getSpecifiers()) {
+                pstmt.clearParameters();
+                if (specifier.getName() != null && !specifier.getName().equals("")) {
+                  pstmt.setString(1, specifier.getName());
+                } else {
+                  pstmt.setString(1, null);
+                }
+                pstmt.setInt(2, tid);
+                pstmt.setString(3, partitions.getPartitionsType().name());
+                pstmt.setInt(4, partitions.getNumPartitions());
+                pstmt.setString(5, columns.toString());
+                pstmt.setString(6, specifier.getExpressions());
+                pstmt.addBatch();
+                count++;
+              }
+            }
+          } else {
+            pstmt.clearParameters();
+            pstmt.setString(1, null);
+            pstmt.setInt(2, tid);
+            pstmt.setString(3, partitions.getPartitionsType().name());
+            pstmt.setInt(4, partitions.getNumPartitions());
+            pstmt.setString(5, columns.toString());
+            pstmt.setString(6, null);
+            pstmt.addBatch();
+          }
+          pstmt.executeBatch();
+        } finally {
+          CatalogUtil.closeSQLWrapper(pstmt);
+        }
+      }
+
     } catch (SQLException se) {
       throw new IOException(se.getMessage(), se);
     } finally {
+      CatalogUtil.closeSQLWrapper(pstmt);
       CatalogUtil.closeSQLWrapper(stmt);
     }
   }
@@ -365,6 +458,20 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       CatalogUtil.closeSQLWrapper(stmt);
     }
 
+
+    try {
+      sql = "DELETE FROM " + TB_PARTTIONS + " WHERE TID IN ("
+        + " SELECT TID FROM " + TB_TABLES
+        + " WHERE " + C_TABLE_ID + " = '" + name + "' )";
+      LOG.info(sql);
+      stmt = getConnection().createStatement();
+      stmt.execute(sql);
+    } catch (SQLException se) {
+      throw new IOException(se);
+    } finally {
+      CatalogUtil.closeSQLWrapper(stmt);
+    }
+
     try {
       sql = "DELETE FROM " + TB_TABLES +
           " WHERE " + C_TABLE_ID + " = '" + name + "'";
@@ -376,6 +483,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     } finally {
       CatalogUtil.closeSQLWrapper(stmt);
     }
+
   }
 
   @Override
@@ -388,10 +496,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     StoreType storeType = null;
     Options options;
     TableStats stat = null;
+    Partitions partitions = null;
+    int tid = 0;
 
     try {
       String sql =
-          "SELECT " + C_TABLE_ID + ", path, store_type from " + TB_TABLES
+          "SELECT " + C_TABLE_ID + ", path, store_type, TID from " + TB_TABLES
               + " WHERE " + C_TABLE_ID + "='" + name + "'";
       if (LOG.isDebugEnabled()) {
         LOG.debug(sql);
@@ -404,6 +514,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       tableName = res.getString(C_TABLE_ID).trim();
       path = new Path(res.getString("path").trim());
       storeType = CatalogUtil.getStoreType(res.getString("store_type").trim());
+      tid = res.getInt("TID");
     } catch (SQLException se) {
       throw new IOException(se);
     } finally {
@@ -481,15 +592,84 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       CatalogUtil.closeSQLWrapper(res, stmt);
     }
 
+    try {
+      String sql = "SELECT name, type, quantity, columns, expressions from " + TB_PARTTIONS
+          + " WHERE TID =" + tid + "";
+      stmt = getConnection().createStatement();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(sql);
+      }
+      res = stmt.executeQuery(sql);
+
+      while (res.next()) {
+        if (partitions == null) {
+          partitions = new Partitions();
+          String[] columns = res.getString("columns").split(",");
+          for(String eachColumn: columns) {
+            partitions.addColumn(getColumn(tableName, tid, eachColumn));
+          }
+          partitions.setPartitionsType(CatalogProtos.PartitionsType.valueOf(res.getString("type")));
+          partitions.setNumPartitions(res.getInt("quantity"));
+        }
+
+        Specifier specifier = new Specifier(res.getString("name"), res.getString("expressions"));
+        partitions.addSpecifier(specifier);
+      }
+
+    } catch (SQLException se) {
+      throw new IOException(se);
+    } finally {
+      CatalogUtil.closeSQLWrapper(res, stmt);
+    }
+
     TableMeta meta = new TableMeta(storeType, options);
     TableDesc table = new TableDesc(tableName, schema, meta, path);
     if (stat != null) {
       table.setStats(stat);
     }
 
+    if (partitions != null) {
+      table.setPartitions(partitions);
+    }
+
     return table;
   }
 
+  private Column getColumn(String tableName, int tid, String columnId) throws IOException {
+    ResultSet res = null;
+    Column column = null;
+    Statement stmt = null;
+
+    try {
+      String sql = "SELECT column_name, data_type, type_length from "
+          + TB_COLUMNS + " WHERE TID = " + tid + " AND column_id = " + columnId;
+
+      stmt = getConnection().createStatement();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(sql);
+      }
+      res = stmt.executeQuery(sql);
+
+      if (res.next()) {
+        String columnName = tableName + "."
+            + res.getString("column_name").trim();
+        Type dataType = getDataType(res.getString("data_type")
+            .trim());
+        int typeLength = res.getInt("type_length");
+        if (typeLength > 0) {
+          column = new Column(columnName, dataType, typeLength);
+        } else {
+          column = new Column(columnName, dataType);
+        }
+      }
+    } catch (SQLException se) {
+      throw new IOException(se);
+    } finally {
+      CatalogUtil.closeSQLWrapper(res, stmt);
+    }
+    return column;
+  }
+
   private Type getDataType(final String typeStr) {
     try {
       return Enum.valueOf(Type.class, typeStr);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
index 6fee78b..06a701b 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
@@ -24,6 +24,9 @@ package org.apache.tajo.catalog.store;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.partition.Partitions;
+import org.apache.tajo.catalog.partition.Specifier;
+import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
@@ -38,6 +41,7 @@ import java.sql.*;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -68,143 +72,177 @@ public class DerbyStore extends AbstractDBStore {
     try {
       // META
       stmt = getConnection().createStatement();
-      String meta_ddl = "CREATE TABLE " + TB_META + " (version int NOT NULL)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(meta_ddl);
+
+      if (!baseTableMaps.get(TB_META)) {
+        String meta_ddl = "CREATE TABLE " + TB_META + " (version int NOT NULL)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(meta_ddl);
+        }
+        stmt.executeUpdate(meta_ddl);
+        LOG.info("Table '" + TB_META + " is created.");
       }
-      stmt.executeUpdate(meta_ddl);
-      LOG.info("Table '" + TB_META + " is created.");
 
       // TABLES
-      String tables_ddl = "CREATE TABLE "
-          + TB_TABLES + " ("
-          + "TID int NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), "
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL CONSTRAINT TABLE_ID_UNIQ UNIQUE, "
-          + "path VARCHAR(1024), "
-          + "store_type CHAR(16), "
-          + "CONSTRAINT TABLES_PK PRIMARY KEY (TID)" +
-          ")";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(tables_ddl);
-      }
-      stmt.addBatch(tables_ddl);
-      String idx_tables_tid = 
-          "CREATE UNIQUE INDEX idx_tables_tid on " + TB_TABLES + " (TID)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_tables_tid);
-      }
-      stmt.addBatch(idx_tables_tid);
-      
-      String idx_tables_name = "CREATE UNIQUE INDEX idx_tables_name on " 
-          + TB_TABLES + "(" + C_TABLE_ID + ")";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_tables_name);
+      if (!baseTableMaps.get(TB_TABLES)) {
+        String tables_ddl = "CREATE TABLE "
+            + TB_TABLES + " ("
+            + "TID int NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), "
+            + C_TABLE_ID + " VARCHAR(255) NOT NULL CONSTRAINT TABLE_ID_UNIQ UNIQUE, "
+            + "path VARCHAR(1024), "
+            + "store_type CHAR(16), "
+            + "CONSTRAINT TABLES_PK PRIMARY KEY (TID)" +
+            ")";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(tables_ddl);
+        }
+        stmt.addBatch(tables_ddl);
+
+        String idx_tables_tid =
+            "CREATE UNIQUE INDEX idx_tables_tid on " + TB_TABLES + " (TID)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_tables_tid);
+        }
+        stmt.addBatch(idx_tables_tid);
+
+        String idx_tables_name = "CREATE UNIQUE INDEX idx_tables_name on "
+            + TB_TABLES + "(" + C_TABLE_ID + ")";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_tables_name);
+        }
+        stmt.addBatch(idx_tables_name);
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_TABLES + "' is created.");
       }
-      stmt.addBatch(idx_tables_name);
-      stmt.executeBatch();
-      LOG.info("Table '" + TB_TABLES + "' is created.");
 
       // COLUMNS
-      String columns_ddl = 
-          "CREATE TABLE " + TB_COLUMNS + " ("
-          + "TID INT NOT NULL REFERENCES " + TB_TABLES + " (TID) ON DELETE CASCADE, "
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES " + TB_TABLES + "("
-          + C_TABLE_ID + ") ON DELETE CASCADE, "
-          + "column_id INT NOT NULL,"
-          + "column_name VARCHAR(255) NOT NULL, " + "data_type CHAR(16), " + "type_length INTEGER, "
-          + "CONSTRAINT C_COLUMN_ID UNIQUE (" + C_TABLE_ID + ", column_name))";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(columns_ddl);
-      }
-      stmt.addBatch(columns_ddl);
+      if (!baseTableMaps.get(TB_COLUMNS)) {
+        String columns_ddl =
+            "CREATE TABLE " + TB_COLUMNS + " ("
+                + "TID INT NOT NULL REFERENCES " + TB_TABLES + " (TID) ON DELETE CASCADE, "
+                + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES " + TB_TABLES + "("
+                + C_TABLE_ID + ") ON DELETE CASCADE, "
+                + "column_id INT NOT NULL,"
+                + "column_name VARCHAR(255) NOT NULL, " + "data_type CHAR(16), " + "type_length INTEGER, "
+                + "CONSTRAINT C_COLUMN_ID UNIQUE (" + C_TABLE_ID + ", column_name))";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(columns_ddl);
+        }
+        stmt.addBatch(columns_ddl);
 
-      String idx_fk_columns_table_name = 
-          "CREATE UNIQUE INDEX idx_fk_columns_table_name on "
-          + TB_COLUMNS + "(" + C_TABLE_ID + ", column_name)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_fk_columns_table_name);
+        String idx_fk_columns_table_name =
+            "CREATE UNIQUE INDEX idx_fk_columns_table_name on "
+                + TB_COLUMNS + "(" + C_TABLE_ID + ", column_name)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_fk_columns_table_name);
+        }
+        stmt.addBatch(idx_fk_columns_table_name);
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_COLUMNS + " is created.");
       }
-      stmt.addBatch(idx_fk_columns_table_name);
-      stmt.executeBatch();
-      LOG.info("Table '" + TB_COLUMNS + " is created.");
 
       // OPTIONS
-      String options_ddl = 
-          "CREATE TABLE " + TB_OPTIONS +" ("
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES TABLES (" + C_TABLE_ID +") "
-          + "ON DELETE CASCADE, "
-          + "key_ VARCHAR(255) NOT NULL, value_ VARCHAR(255) NOT NULL)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(options_ddl);
-      }
-      stmt.addBatch(options_ddl);
-      
-      String idx_options_key = 
-          "CREATE INDEX idx_options_key on " + TB_OPTIONS + " (" + C_TABLE_ID + ")";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_options_key);
-      }
-      stmt.addBatch(idx_options_key);
-      String idx_options_table_name = 
-          "CREATE INDEX idx_options_table_name on " + TB_OPTIONS 
-          + "(" + C_TABLE_ID + ")";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_options_table_name);
+      if (!baseTableMaps.get(TB_OPTIONS)) {
+        String options_ddl =
+            "CREATE TABLE " + TB_OPTIONS +" ("
+                + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES TABLES (" + C_TABLE_ID +") "
+                + "ON DELETE CASCADE, "
+                + "key_ VARCHAR(255) NOT NULL, value_ VARCHAR(255) NOT NULL)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(options_ddl);
+        }
+        stmt.addBatch(options_ddl);
+
+        String idx_options_key =
+            "CREATE INDEX idx_options_key on " + TB_OPTIONS + " (" + C_TABLE_ID + ")";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_options_key);
+        }
+        stmt.addBatch(idx_options_key);
+        String idx_options_table_name =
+            "CREATE INDEX idx_options_table_name on " + TB_OPTIONS
+                + "(" + C_TABLE_ID + ")";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_options_table_name);
+        }
+        stmt.addBatch(idx_options_table_name);
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_OPTIONS + " is created.");
       }
-      stmt.addBatch(idx_options_table_name);
-      stmt.executeBatch();
-      LOG.info("Table '" + TB_OPTIONS + " is created.");
-      
+
       // INDEXES
-      String indexes_ddl = "CREATE TABLE " + TB_INDEXES +"("
-          + "index_name VARCHAR(255) NOT NULL PRIMARY KEY, "
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES TABLES (" + C_TABLE_ID + ") "
-          + "ON DELETE CASCADE, "
-          + "column_name VARCHAR(255) NOT NULL, "
-          + "data_type VARCHAR(255) NOT NULL, "
-          + "index_type CHAR(32) NOT NULL, "
-          + "is_unique BOOLEAN NOT NULL, "
-          + "is_clustered BOOLEAN NOT NULL, "
-          + "is_ascending BOOLEAN NOT NULL)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(indexes_ddl);
+      if (!baseTableMaps.get(TB_INDEXES)) {
+        String indexes_ddl = "CREATE TABLE " + TB_INDEXES +"("
+            + "index_name VARCHAR(255) NOT NULL PRIMARY KEY, "
+            + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES TABLES (" + C_TABLE_ID + ") "
+            + "ON DELETE CASCADE, "
+            + "column_name VARCHAR(255) NOT NULL, "
+            + "data_type VARCHAR(255) NOT NULL, "
+            + "index_type CHAR(32) NOT NULL, "
+            + "is_unique BOOLEAN NOT NULL, "
+            + "is_clustered BOOLEAN NOT NULL, "
+            + "is_ascending BOOLEAN NOT NULL)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(indexes_ddl);
+        }
+        stmt.addBatch(indexes_ddl);
+
+        String idx_indexes_key = "CREATE UNIQUE INDEX idx_indexes_key ON "
+            + TB_INDEXES + " (index_name)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_indexes_key);
+        }
+        stmt.addBatch(idx_indexes_key);
+
+        String idx_indexes_columns = "CREATE INDEX idx_indexes_columns ON "
+            + TB_INDEXES + " (" + C_TABLE_ID + ", column_name)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_indexes_columns);
+        }
+        stmt.addBatch(idx_indexes_columns);
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_INDEXES + "' is created.");
       }
-      stmt.addBatch(indexes_ddl);
-      
-      String idx_indexes_key = "CREATE UNIQUE INDEX idx_indexes_key ON " 
-          + TB_INDEXES + " (index_name)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_indexes_key);
-      }      
-      stmt.addBatch(idx_indexes_key);
-      
-      String idx_indexes_columns = "CREATE INDEX idx_indexes_columns ON " 
-          + TB_INDEXES + " (" + C_TABLE_ID + ", column_name)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_indexes_columns);
-      } 
-      stmt.addBatch(idx_indexes_columns);
-      stmt.executeBatch();
-      LOG.info("Table '" + TB_INDEXES + "' is created.");
 
-      String stats_ddl = "CREATE TABLE " + TB_STATISTICS + "("
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES TABLES (" + C_TABLE_ID + ") "
-          + "ON DELETE CASCADE, "
-          + "num_rows BIGINT, "
-          + "num_bytes BIGINT)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(stats_ddl);
+      if (!baseTableMaps.get(TB_STATISTICS)) {
+        String stats_ddl = "CREATE TABLE " + TB_STATISTICS + "("
+            + C_TABLE_ID + " VARCHAR(255) NOT NULL REFERENCES TABLES (" + C_TABLE_ID + ") "
+            + "ON DELETE CASCADE, "
+            + "num_rows BIGINT, "
+            + "num_bytes BIGINT)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(stats_ddl);
+        }
+        stmt.addBatch(stats_ddl);
+
+        String idx_stats_fk_table_name = "CREATE INDEX idx_stats_table_name ON "
+            + TB_STATISTICS + " (" + C_TABLE_ID + ")";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(idx_stats_fk_table_name);
+        }
+        stmt.addBatch(idx_stats_fk_table_name);
+        LOG.info("Table '" + TB_STATISTICS + "' is created.");
       }
-      stmt.addBatch(stats_ddl);
 
-      String idx_stats_fk_table_name = "CREATE INDEX idx_stats_table_name ON "
-          + TB_STATISTICS + " (" + C_TABLE_ID + ")";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(idx_stats_fk_table_name);
+      // PARTITION
+      if (!baseTableMaps.get(TB_PARTTIONS)) {
+        String partition_ddl = "CREATE TABLE " + TB_PARTTIONS + " ("
+            + "PID INT NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), "
+            + "name VARCHAR(255), "
+            + "TID INT NOT NULL REFERENCES " + TB_TABLES + " (TID) ON DELETE CASCADE, "
+            + "type VARCHAR(10) NOT NULL,"
+            + "quantity INT ,"
+            + "columns VARCHAR(255),"
+            + "expressions VARCHAR(1024)"
+            + ", CONSTRAINT PARTITION_PK PRIMARY KEY (PID)"
+            + "   )";
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(partition_ddl);
+        }
+        stmt.addBatch(partition_ddl);
+        LOG.info("Table '" + TB_PARTTIONS + "' is created.");
+        stmt.executeBatch();
       }
-      stmt.addBatch(idx_stats_fk_table_name);
-      stmt.executeBatch();
-      LOG.info("Table '" + TB_STATISTICS + "' is created.");
 
     } finally {
       wlock.unlock();
@@ -220,26 +258,34 @@ public class DerbyStore extends AbstractDBStore {
 
     wlock.lock();
     ResultSet res = null;
+    int foundCount = 0;
     try {
-      boolean found = false;
       res = getConnection().getMetaData().getTables(null, null, null,
           new String [] {"TABLE"});
-      
-      String resName;
-      while (res.next() && !found) {
-        resName = res.getString("TABLE_NAME");
-        if (TB_META.equals(resName)
-            || TB_TABLES.equals(resName)
-            || TB_COLUMNS.equals(resName)
-            || TB_OPTIONS.equals(resName)) {
-            return true;
-        }
+
+      baseTableMaps.put(TB_META, false);
+      baseTableMaps.put(TB_TABLES, false);
+      baseTableMaps.put(TB_COLUMNS, false);
+      baseTableMaps.put(TB_OPTIONS, false);
+      baseTableMaps.put(TB_STATISTICS, false);
+      baseTableMaps.put(TB_INDEXES, false);
+      baseTableMaps.put(TB_PARTTIONS, false);
+
+      while (res.next()) {
+        baseTableMaps.put(res.getString("TABLE_NAME"), true);
       }
     } finally {
       wlock.unlock();
       CatalogUtil.closeSQLWrapper(res);
-    }    
-    return false;
+    }
+
+    for(Map.Entry<String, Boolean> entry : baseTableMaps.entrySet()) {
+      if (!entry.getValue()) {
+        return false;
+      }
+    }
+
+    return true;
   }
   
   final boolean checkInternalTable(final String tableName) throws SQLException {
@@ -263,6 +309,7 @@ public class DerbyStore extends AbstractDBStore {
   
   @Override
   public final void addTable(final TableDesc table) throws IOException {
+    PreparedStatement pstmt = null;
     Statement stmt = null;
     ResultSet res = null;
 
@@ -324,10 +371,95 @@ public class DerbyStore extends AbstractDBStore {
         }
         stmt.executeUpdate(sql);
       }
+
+      //Partition
+      if (table.getPartitions() != null && !table.getPartitions().toString().isEmpty()) {
+        try {
+          Partitions partitions = table.getPartitions();
+          List<Column> columnList = partitions.getColumns();
+
+          // Find columns which used for a partitioned table.
+          StringBuffer columns = new StringBuffer();
+          for(Column eachColumn : columnList) {
+            sql = "SELECT column_id from " + TB_COLUMNS + " WHERE TID "
+                + " = " + tid + " AND column_name = '" + eachColumn.getColumnName() + "'";
+
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(sql);
+            }
+            res = stmt.executeQuery(sql);
+            if (!res.next()) {
+              throw new IOException("ERROR: there is no columnId matched to "
+                  + table.getName());
+            }
+            columnId = res.getInt("column_id");
+
+            if(columns.length() > 0) {
+              columns.append(",");
+            }
+            columns.append(columnId);
+          }
+
+          // Set default partition name. But if user named to subpartition, it would be updated.
+//          String partitionName = partitions.getPartitionsType().name() + "_" + table.getName();
+
+          sql = "INSERT INTO " + TB_PARTTIONS + " (name, TID, "
+              + " type, quantity, columns, expressions) VALUES (?, ?, ?, ?, ?, ?) ";
+          pstmt = getConnection().prepareStatement(sql);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(sql);
+          }
+
+          // Find information for subpartitions
+          if (partitions.getSpecifiers() != null) {
+            int count = 1;
+            if (partitions.getSpecifiers().size() == 0) {
+              pstmt.clearParameters();
+              pstmt.setString(1, null);
+              pstmt.setInt(2, tid);
+              pstmt.setString(3, partitions.getPartitionsType().name());
+              pstmt.setInt(4, partitions.getNumPartitions());
+              pstmt.setString(5, columns.toString());
+              pstmt.setString(6, null);
+              pstmt.addBatch();
+            } else {
+              for(Specifier eachValue: partitions.getSpecifiers()) {
+                pstmt.clearParameters();
+                if (eachValue.getName() != null && !eachValue.getName().equals("")) {
+                  pstmt.setString(1, eachValue.getName());
+                } else {
+                  pstmt.setString(1, null);
+                }
+                pstmt.setInt(2, tid);
+                pstmt.setString(3, partitions.getPartitionsType().name());
+                pstmt.setInt(4, partitions.getNumPartitions());
+                pstmt.setString(5, columns.toString());
+                pstmt.setString(6, eachValue.getExpressions());
+                pstmt.addBatch();
+                count++;
+              }
+            }
+          } else {
+            pstmt.clearParameters();
+            pstmt.setString(1, null);
+            pstmt.setInt(2, tid);
+            pstmt.setString(3, partitions.getPartitionsType().name());
+            pstmt.setInt(4, partitions.getNumPartitions());
+            pstmt.setString(5, columns.toString());
+            pstmt.setString(6, null);
+            pstmt.addBatch();
+          }
+          pstmt.executeBatch();
+        } finally {
+          CatalogUtil.closeSQLWrapper(pstmt);
+        }
+      }
+
     } catch (SQLException se) {
       throw new IOException(se.getMessage(), se);
     } finally {
       wlock.unlock();
+      CatalogUtil.closeSQLWrapper(res, pstmt);
       CatalogUtil.closeSQLWrapper(res, stmt);
     }
   }
@@ -428,6 +560,17 @@ public class DerbyStore extends AbstractDBStore {
       }
 
       try {
+        sql = "DELETE FROM " + TB_PARTTIONS + " WHERE TID IN ("
+            + " SELECT TID FROM " + TB_TABLES
+            + " WHERE " + C_TABLE_ID + " = '" + name + "' )";
+        LOG.info(sql);
+        stmt = getConnection().createStatement();
+        stmt.execute(sql);
+      } catch (SQLException se) {
+        throw new IOException(se);
+      }
+
+      try {
         sql = "DELETE FROM " + TB_TABLES +
             " WHERE " + C_TABLE_ID +" = '" + name + "'";
         LOG.info(sql);
@@ -435,7 +578,6 @@ public class DerbyStore extends AbstractDBStore {
       } catch (SQLException se) {
         throw new IOException(se);
       }
-
     } catch (SQLException se) {
       throw new IOException(se);
     } finally {
@@ -454,6 +596,8 @@ public class DerbyStore extends AbstractDBStore {
     StoreType storeType = null;
     Options options;
     TableStats stat = null;
+    Partitions partitions = null;
+    int tid = 0;
 
     try {
       rlock.lock();
@@ -461,7 +605,7 @@ public class DerbyStore extends AbstractDBStore {
 
       try {
         String sql = 
-            "SELECT " + C_TABLE_ID + ", path, store_type from " + TB_TABLES
+            "SELECT " + C_TABLE_ID + ", path, store_type, TID from " + TB_TABLES
             + " WHERE " + C_TABLE_ID + "='" + name + "'";
         if (LOG.isDebugEnabled()) {
           LOG.debug(sql);
@@ -474,6 +618,7 @@ public class DerbyStore extends AbstractDBStore {
         tableName = res.getString(C_TABLE_ID).trim();
         path = new Path(res.getString("path").trim());
         storeType = CatalogUtil.getStoreType(res.getString("store_type").trim());
+        tid = res.getInt("TID");
       } catch (SQLException se) { 
         throw new IOException(se);
       } finally {
@@ -550,12 +695,48 @@ public class DerbyStore extends AbstractDBStore {
         CatalogUtil.closeSQLWrapper(res);
       }
 
+
+      try {
+        String sql = "SELECT name, type, quantity, columns, expressions from " + TB_PARTTIONS
+            + " WHERE TID =" + tid + "";
+        stmt = getConnection().createStatement();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        res = stmt.executeQuery(sql);
+
+        while (res.next()) {
+          if (partitions == null) {
+            partitions = new Partitions();
+            String[] columns = res.getString("columns").split(",");
+            for(String eachColumn: columns) {
+              partitions.addColumn(getColumn(tableName, tid, eachColumn));
+            }
+            partitions.setPartitionsType(CatalogProtos.PartitionsType.valueOf(res.getString
+                ("type")));
+            partitions.setNumPartitions(res.getInt("quantity"));
+          }
+
+          Specifier specifier = new Specifier(res.getString("name"), res.getString("expressions"));
+          partitions.addSpecifier(specifier);
+        }
+
+      } catch (SQLException se) {
+        throw new IOException(se);
+      } finally {
+        CatalogUtil.closeSQLWrapper(res, stmt);
+      }
+
       TableMeta meta = new TableMeta(storeType, options);
       TableDesc table = new TableDesc(tableName, schema, meta, path);
       if (stat != null) {
         table.setStats(stat);
       }
 
+      if (partitions != null) {
+        table.setPartitions(partitions);
+      }
+
       return table;
     } catch (SQLException se) {
       throw new IOException(se);
@@ -564,7 +745,42 @@ public class DerbyStore extends AbstractDBStore {
       CatalogUtil.closeSQLWrapper(stmt);
     }
   }
-  
+
+  private Column getColumn(String tableName, int tid, String columnId) throws IOException {
+    ResultSet res = null;
+    Column column = null;
+    Statement stmt = null;
+
+    try {
+      String sql = "SELECT column_name, data_type, type_length from "
+          + TB_COLUMNS + " WHERE TID = " + tid + " AND column_id = " + columnId;
+
+      stmt = getConnection().createStatement();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(sql);
+      }
+      res = stmt.executeQuery(sql);
+
+      if (res.next()) {
+        String columnName = tableName + "."
+            + res.getString("column_name").trim();
+        Type dataType = getDataType(res.getString("data_type")
+            .trim());
+        int typeLength = res.getInt("type_length");
+        if (typeLength > 0) {
+          column = new Column(columnName, dataType, typeLength);
+        } else {
+          column = new Column(columnName, dataType);
+        }
+      }
+    } catch (SQLException se) {
+      throw new IOException(se);
+    } finally {
+      CatalogUtil.closeSQLWrapper(res, stmt);
+    }
+    return column;
+  }
+
   private Type getDataType(final String typeStr) {
     try {
     return Enum.valueOf(Type.class, typeStr);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
index 259d9d6..c368969 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
@@ -29,6 +29,7 @@ import org.apache.tajo.exception.InternalException;
 import java.io.IOException;
 import java.sql.*;
 import java.util.List;
+import java.util.Map;
 
 public class MySQLStore extends AbstractDBStore  {
 
@@ -51,115 +52,158 @@ public class MySQLStore extends AbstractDBStore  {
   // TODO - DDL and index statements should be renamed
   protected void createBaseTable() throws SQLException {
 
-    // META
-    Statement stmt = getConnection().createStatement();
-    String meta_ddl = "CREATE TABLE " + TB_META + " (version int NOT NULL)";
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(meta_ddl);
-    }
+    int result;
+    Statement stmt = null;
     try {
-      int result = stmt.executeUpdate(meta_ddl);
-      LOG.info("Table '" + TB_META + " is created.");
+      stmt = getConnection().createStatement();
+
+      // META
+      if (!baseTableMaps.get(TB_META)) {
+        String meta_ddl = "CREATE TABLE " + TB_META + " (version int NOT NULL)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(meta_ddl);
+        }
+        result = stmt.executeUpdate(meta_ddl);
+        LOG.info("Table '" + TB_META + " is created.");
+      }
 
       // TABLES
-      String tables_ddl = "CREATE TABLE "
-          + TB_TABLES + " ("
-          + "TID int NOT NULL AUTO_INCREMENT PRIMARY KEY, "
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL UNIQUE, "
-          + "path TEXT, "
-          + "store_type CHAR(16)"
-          + ")";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(tables_ddl);
+      if (!baseTableMaps.get(TB_TABLES)) {
+        String tables_ddl = "CREATE TABLE "
+            + TB_TABLES + " ("
+            + "TID int NOT NULL AUTO_INCREMENT PRIMARY KEY, "
+            + C_TABLE_ID + " VARCHAR(255) NOT NULL UNIQUE, "
+            + "path TEXT, "
+            + "store_type CHAR(16)"
+            + ")";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(tables_ddl);
+        }
+
+        LOG.info("Table '" + TB_TABLES + "' is created.");
+        result = stmt.executeUpdate(tables_ddl);
       }
 
-      LOG.info("Table '" + TB_TABLES + "' is created.");
-      result = stmt.executeUpdate(tables_ddl);
       // COLUMNS
+      if (!baseTableMaps.get(TB_COLUMNS)) {
+        String columns_ddl =
+            "CREATE TABLE " + TB_COLUMNS + " ("
+                + "TID INT NOT NULL,"
+                + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
+                + "column_id INT NOT NULL,"
+                + "column_name VARCHAR(255) NOT NULL, " + "data_type CHAR(16), " + "type_length INTEGER, "
+                + "UNIQUE KEY(" + C_TABLE_ID + ", column_name),"
+                + "FOREIGN KEY(TID) REFERENCES "+TB_TABLES+"(TID) ON DELETE CASCADE,"
+                + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(columns_ddl);
+        }
 
-      String columns_ddl =
-          "CREATE TABLE " + TB_COLUMNS + " ("
-              + "TID INT NOT NULL,"
-              + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
-              + "column_id INT NOT NULL,"
-              + "column_name VARCHAR(255) NOT NULL, " + "data_type CHAR(16), " + "type_length INTEGER, "
-              + "UNIQUE KEY(" + C_TABLE_ID + ", column_name),"
-              + "FOREIGN KEY(TID) REFERENCES "+TB_TABLES+"(TID) ON DELETE CASCADE,"
-              + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(columns_ddl);
+        LOG.info("Table '" + TB_COLUMNS + " is created.");
+        result = stmt.executeUpdate(columns_ddl);
       }
 
-      LOG.info("Table '" + TB_COLUMNS + " is created.");
-      result = stmt.executeUpdate(columns_ddl);
       // OPTIONS
-
-      String options_ddl =
-          "CREATE TABLE " + TB_OPTIONS + " ("
-              + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
-              + "key_ VARCHAR(255) NOT NULL, value_ VARCHAR(255) NOT NULL,"
-              + "INDEX("+C_TABLE_ID+", key_),"
-              + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(options_ddl);
+      if (!baseTableMaps.get(TB_OPTIONS)) {
+        String options_ddl =
+            "CREATE TABLE " + TB_OPTIONS + " ("
+                + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
+                + "key_ VARCHAR(255) NOT NULL, value_ VARCHAR(255) NOT NULL,"
+                + "INDEX("+C_TABLE_ID+", key_),"
+                + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(options_ddl);
+        }
+        LOG.info("Table '" + TB_OPTIONS + " is created.");
+        result = stmt.executeUpdate(options_ddl);
       }
-      LOG.info("Table '" + TB_OPTIONS + " is created.");
-      result = stmt.executeUpdate(options_ddl);
+
       // INDEXES
+      if (!baseTableMaps.get(TB_INDEXES)) {
+        String indexes_ddl = "CREATE TABLE " + TB_INDEXES + "("
+            + "index_name VARCHAR(255) NOT NULL PRIMARY KEY, "
+            + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
+            + "column_name VARCHAR(255) NOT NULL, "
+            + "data_type VARCHAR(255) NOT NULL, "
+            + "index_type CHAR(32) NOT NULL, "
+            + "is_unique BOOLEAN NOT NULL, "
+            + "is_clustered BOOLEAN NOT NULL, "
+            + "is_ascending BOOLEAN NOT NULL,"
+            + "INDEX(" + C_TABLE_ID + ", column_name),"
+            + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(indexes_ddl);
+        }
+        LOG.info("Table '" + TB_INDEXES + "' is created.");
+        result = stmt.executeUpdate(indexes_ddl);
+      }
 
-      String indexes_ddl = "CREATE TABLE " + TB_INDEXES + "("
-          + "index_name VARCHAR(255) NOT NULL PRIMARY KEY, "
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
-          + "column_name VARCHAR(255) NOT NULL, "
-          + "data_type VARCHAR(255) NOT NULL, "
-          + "index_type CHAR(32) NOT NULL, "
-          + "is_unique BOOLEAN NOT NULL, "
-          + "is_clustered BOOLEAN NOT NULL, "
-          + "is_ascending BOOLEAN NOT NULL,"
-          + "INDEX(" + C_TABLE_ID + ", column_name),"
-          + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(indexes_ddl);
+      if (!baseTableMaps.get(TB_STATISTICS)) {
+        String stats_ddl = "CREATE TABLE " + TB_STATISTICS + "("
+            + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
+            + "num_rows BIGINT, "
+            + "num_bytes BIGINT,"
+            + "INDEX("+C_TABLE_ID+"),"
+            + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(stats_ddl);
+        }
+        LOG.info("Table '" + TB_STATISTICS + "' is created.");
+        result = stmt.executeUpdate(stats_ddl);
       }
-      LOG.info("Table '" + TB_INDEXES + "' is created.");
-      result = stmt.executeUpdate(indexes_ddl);
-
-      String stats_ddl = "CREATE TABLE " + TB_STATISTICS + "("
-          + C_TABLE_ID + " VARCHAR(255) NOT NULL,"
-          + "num_rows BIGINT, "
-          + "num_bytes BIGINT,"
-          + "INDEX("+C_TABLE_ID+"),"
-          + "FOREIGN KEY("+C_TABLE_ID+") REFERENCES "+TB_TABLES+"("+C_TABLE_ID+") ON DELETE CASCADE)";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(stats_ddl);
+
+      // PARTITION
+      if (!baseTableMaps.get(TB_PARTTIONS)) {
+        String partition_ddl = "CREATE TABLE " + TB_PARTTIONS + " ("
+            + "PID int NOT NULL AUTO_INCREMENT PRIMARY KEY, "
+            + "name VARCHAR(255), "
+            + "TID INT NOT NULL,"
+            + "type VARCHAR(10) NOT NULL,"
+            + "quantity INT ,"
+            + "columns VARCHAR(255),"
+            + "expressions TEXT )";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(partition_ddl);
+        }
+        LOG.info("Table '" + TB_PARTTIONS + "' is created.");
+        result = stmt.executeUpdate(partition_ddl);
       }
-      LOG.info("Table '" + TB_STATISTICS + "' is created.");
-      result = stmt.executeUpdate(stats_ddl);
     } finally {
       CatalogUtil.closeSQLWrapper(stmt);
     }
   }
 
   protected boolean isInitialized() throws SQLException {
-    boolean found = false;
     ResultSet res = getConnection().getMetaData().getTables(null, null, null,
         new String[]{"TABLE"});
 
-    String resName;
     try {
-      while (res.next() && !found) {
-        resName = res.getString("TABLE_NAME");
-        if (TB_META.equals(resName)
-            || TB_TABLES.equals(resName)
-            || TB_COLUMNS.equals(resName)
-            || TB_OPTIONS.equals(resName)) {
-          return true;
-        }
+      baseTableMaps.put(TB_META, false);
+      baseTableMaps.put(TB_TABLES, false);
+      baseTableMaps.put(TB_COLUMNS, false);
+      baseTableMaps.put(TB_OPTIONS, false);
+      baseTableMaps.put(TB_STATISTICS, false);
+      baseTableMaps.put(TB_INDEXES, false);
+      baseTableMaps.put(TB_PARTTIONS, false);
+
+      if (res.wasNull())
+        return false;
+
+      while (res.next()) {
+        baseTableMaps.put(res.getString("TABLE_NAME"), true);
       }
     } finally {
       CatalogUtil.closeSQLWrapper(res);
     }
-    return false;
+
+    for(Map.Entry<String, Boolean> entry : baseTableMaps.entrySet()) {
+      if (!entry.getValue()) {
+        return false;
+      }
+    }
+
+    return  true;
+//    return false;
   }
 
   @Override


[18/18] git commit: Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/incubator-tajo into DAG-execplan

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


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

Branch: refs/heads/DAG-execplan
Commit: dc24dbc1e8950c4ae88abc2635b275d211b16644
Parents: 0cbd968 abc0830
Author: Jihoon Son <ji...@apache.org>
Authored: Tue Dec 3 20:50:20 2013 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Tue Dec 3 20:50:20 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  42 +-
 .../org/apache/tajo/algebra/CreateTable.java    |  11 +-
 .../org/apache/tajo/algebra/FunctionExpr.java   |   4 +-
 .../apache/tajo/catalog/CatalogConstants.java   |   1 +
 .../org/apache/tajo/catalog/CatalogUtil.java    |   3 +
 .../org/apache/tajo/catalog/DDLBuilder.java     |  70 +++
 .../java/org/apache/tajo/catalog/TableDesc.java |  33 +-
 .../tajo/catalog/partition/Partitions.java      | 349 +++++++++++++
 .../tajo/catalog/partition/Specifier.java       | 128 +++++
 .../src/main/proto/CatalogProtos.proto          |  20 +
 tajo-catalog/tajo-catalog-server/pom.xml        |   4 +
 .../org/apache/tajo/catalog/CatalogServer.java  |   5 +-
 .../tajo/catalog/store/AbstractDBStore.java     | 184 ++++++-
 .../apache/tajo/catalog/store/DerbyStore.java   | 488 +++++++++++++------
 .../apache/tajo/catalog/store/MySQLStore.java   | 204 +++++---
 .../org/apache/tajo/catalog/TestCatalog.java    | 225 ++++++++-
 .../org/apache/tajo/catalog/TestDBStore.java    | 174 +++++++
 .../java/org/apache/tajo/conf/TajoConf.java     |  14 +-
 .../java/org/apache/tajo/datum/TextDatum.java   |   2 +
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  12 +-
 .../main/java/org/apache/tajo/cli/TajoCli.java  |  36 ++
 .../apache/tajo/engine/function/math/Acos.java  |  50 ++
 .../apache/tajo/engine/function/math/Asin.java  |  50 ++
 .../apache/tajo/engine/function/math/Atan.java  |  50 ++
 .../apache/tajo/engine/function/math/Atan2.java |  51 ++
 .../apache/tajo/engine/function/math/Ceil.java  |  50 ++
 .../apache/tajo/engine/function/math/Cos.java   |  50 ++
 .../apache/tajo/engine/function/math/Floor.java |  50 ++
 .../apache/tajo/engine/function/math/Round.java |  50 ++
 .../apache/tajo/engine/function/math/Sin.java   |  50 ++
 .../apache/tajo/engine/function/math/Tan.java   |  51 ++
 .../tajo/engine/function/string/InitCap.java    |  49 ++
 .../tajo/engine/function/string/Left.java       |  73 +++
 .../apache/tajo/engine/function/string/Md5.java |  55 +++
 .../engine/function/string/OctetLength.java     |  51 ++
 .../tajo/engine/function/string/Repeat.java     |  61 +++
 .../tajo/engine/function/string/Reverse.java    |  48 ++
 .../tajo/engine/function/string/Right.java      |  74 +++
 .../tajo/engine/function/string/StrPos.java     |  62 +++
 .../tajo/engine/function/string/StrPosb.java    | 109 +++++
 .../tajo/engine/function/string/ToHex.java      |  68 +++
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |   9 +-
 .../apache/tajo/engine/planner/LogicalPlan.java |  13 +-
 .../tajo/engine/planner/LogicalPlanner.java     | 135 ++++-
 .../engine/planner/logical/CreateTableNode.java |  18 +-
 .../engine/planner/logical/StoreTableNode.java  |  26 +-
 .../org/apache/tajo/master/GlobalEngine.java    |   9 +-
 .../apache/tajo/master/TajoContainerProxy.java  |  38 +-
 .../java/org/apache/tajo/master/TajoMaster.java | 153 ++++++
 .../tajo/master/TajoMasterClientService.java    |   6 +-
 .../apache/tajo/master/TajoMasterService.java   |  25 +-
 .../tajo/master/querymaster/SubQuery.java       |   9 +-
 .../tajo/master/rm/TajoWorkerContainerId.java   |  41 ++
 .../master/rm/TajoWorkerResourceManager.java    | 358 +++++++++++---
 .../apache/tajo/master/rm/WorkerResource.java   | 115 ++---
 .../tajo/master/rm/WorkerResourceManager.java   |   6 +-
 .../tajo/master/rm/YarnTajoResourceManager.java |  28 +-
 .../apache/tajo/util/ApplicationIdUtils.java    |   3 +
 .../apache/tajo/worker/ResourceAllocator.java   |   3 +-
 .../tajo/worker/TajoResourceAllocator.java      |  68 +--
 .../java/org/apache/tajo/worker/TajoWorker.java |  86 ++--
 .../tajo/worker/YarnResourceAllocator.java      |   4 +-
 .../src/main/proto/ClientProtos.proto           |   1 +
 .../src/main/proto/TajoMasterProtocol.proto     |  64 ++-
 .../main/resources/webapps/admin/cluster.jsp    |  31 +-
 .../src/main/resources/webapps/admin/index.jsp  |  22 +-
 .../org/apache/tajo/TajoTestingCluster.java     |   3 +
 .../org/apache/tajo/client/TestTajoClient.java  | 122 +++++
 .../tajo/engine/function/TestMathFunctions.java | 199 ++++++++
 .../TestStringOperatorsAndFunctions.java        | 169 +++++++
 .../tajo/master/TestTajoResourceManager.java    | 390 +++++++++++++++
 71 files changed, 4761 insertions(+), 554 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/dc24dbc1/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/dc24dbc1/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/dc24dbc1/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
----------------------------------------------------------------------
diff --cc tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
index 7eeb3b7,b2bd937..dddef1b
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
@@@ -169,14 -186,15 +186,19 @@@ public class StoreTableNode extends Una
      }
      
      sb.append("\n  \"out schema\": ").append(getOutSchema()).append(",")
-     .append("\n  \"in schema\": ").append(getInSchema())
-     .append("}");
+     .append("\n  \"in schema\": ").append(getInSchema());
+ 
+     if(partitions != null) {
+       sb.append(partitions.toString());
+     }
+ 
+     sb.append("}");
      
 -    return sb.toString() + "\n"
 +    if (child != null) {
 +      return sb.toString() + "\n"
          + getChild().toString();
 +    } else {
 +      return sb.toString();
 +    }
    }
  }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/dc24dbc1/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------


[14/18] TAJO-284: Add table partitioning entry to Catalog. (jaehwa)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
index b824756..d174e72 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
@@ -20,6 +20,9 @@ package org.apache.tajo.catalog;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.function.Function;
+import org.apache.tajo.catalog.partition.Partitions;
+import org.apache.tajo.catalog.partition.Specifier;
+import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
@@ -70,19 +73,19 @@ public class TestCatalog {
 		schema1.addColumn(FieldName2, Type.INT4);
 		schema1.addColumn(FieldName3, Type.INT8);
     Path path = new Path(CommonTestingUtil.getTestDir(), "table1");
-		TableDesc meta = CatalogUtil.newTableDesc(
+    TableDesc meta = CatalogUtil.newTableDesc(
         "getTable",
         schema1,
         StoreType.CSV,
         new Options(),
         path);
-		
-		assertFalse(catalog.existsTable("getTable"));
-		catalog.addTable(meta);
-		assertTrue(catalog.existsTable("getTable"));
-		
-		catalog.deleteTable("getTable");
+
 		assertFalse(catalog.existsTable("getTable"));
+    catalog.addTable(meta);
+    assertTrue(catalog.existsTable("getTable"));
+
+    catalog.deleteTable("getTable");
+    assertFalse(catalog.existsTable("getTable"));
 	}
 	
 	@Test(expected = Throwable.class)
@@ -197,4 +200,212 @@ public class TestCatalog {
     catalog.createFunction(overload);
     assertTrue(catalog.containFunction("test3", CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.BLOB)));
   }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByHash1() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.HASH);
+    partitions.setNumPartitions(2);
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+
+    assertFalse(catalog.existsTable(tableName));
+    catalog.addTable(desc);
+    assertTrue(catalog.existsTable(tableName));
+    TableDesc retrieved = catalog.getTableDesc(tableName);
+
+    assertEquals(retrieved.getName(), tableName);
+    assertEquals(retrieved.getPartitions().getPartitionsType(), CatalogProtos.PartitionsType.HASH);
+    assertEquals(retrieved.getPartitions().getColumn(0).getColumnName(), "id");
+    assertEquals(retrieved.getPartitions().getNumPartitions(), 2);
+
+    catalog.deleteTable(tableName);
+    assertFalse(catalog.existsTable(tableName));
+  }
+
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByHash2() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.HASH);
+    partitions.setNumPartitions(2);
+
+    partitions.addSpecifier(new Specifier("sub_part1"));
+    partitions.addSpecifier(new Specifier("sub_part2"));
+    partitions.addSpecifier(new Specifier("sub_part3"));
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(catalog.existsTable(tableName));
+    catalog.addTable(desc);
+    assertTrue(catalog.existsTable(tableName));
+
+    TableDesc retrieved = catalog.getTableDesc(tableName);
+
+    assertEquals(retrieved.getName(), tableName);
+    assertEquals(retrieved.getPartitions().getPartitionsType(), CatalogProtos.PartitionsType.HASH);
+    assertEquals(retrieved.getPartitions().getColumn(0).getColumnName(), "id");
+    assertEquals(retrieved.getPartitions().getNumPartitions(), 2);
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(0).getName(),
+        "sub_part1");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(1).getName(),
+        "sub_part2");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(2).getName(),
+        "sub_part3");
+
+    catalog.deleteTable(tableName);
+    assertFalse(catalog.existsTable(tableName));
+  }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByList() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.LIST);
+
+    partitions.addSpecifier(new Specifier("sub_part1", "Seoul,서울"));
+    partitions.addSpecifier(new Specifier("sub_part2", "Busan,부산"));
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(catalog.existsTable(tableName));
+    catalog.addTable(desc);
+    assertTrue(catalog.existsTable(tableName));
+
+    TableDesc retrieved = catalog.getTableDesc(tableName);
+
+    assertEquals(retrieved.getName(), tableName);
+    assertEquals(retrieved.getPartitions().getPartitionsType(), CatalogProtos.PartitionsType.LIST);
+    assertEquals(retrieved.getPartitions().getColumn(0).getColumnName(), "id");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(0).getName(),
+        "sub_part1");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(0).getExpressions(),
+        "Seoul,서울");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(1).getName(),
+        "sub_part2");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(1).getExpressions(),
+        "Busan,부산");
+
+    catalog.deleteTable(tableName);
+    assertFalse(catalog.existsTable(tableName));
+  }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByRange() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.RANGE);
+
+    partitions.addSpecifier(new Specifier("sub_part1", "2"));
+    partitions.addSpecifier(new Specifier("sub_part2", "5"));
+    partitions.addSpecifier(new Specifier("sub_part3"));
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(catalog.existsTable(tableName));
+    catalog.addTable(desc);
+    assertTrue(catalog.existsTable(tableName));
+
+    TableDesc retrieved = catalog.getTableDesc(tableName);
+
+    assertEquals(retrieved.getName(), tableName);
+    assertEquals(retrieved.getPartitions().getPartitionsType(), CatalogProtos.PartitionsType.RANGE);
+    assertEquals(retrieved.getPartitions().getColumn(0).getColumnName(), "id");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(0).getName(),
+        "sub_part1");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(0).getExpressions(),
+        "2");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(1).getName(),
+        "sub_part2");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(1).getExpressions(),
+        "5");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(2).getName(),
+        "sub_part3");
+    assertEquals(retrieved.getPartitions().getSpecifiers().get(2).getExpressions(),
+        "");
+
+    catalog.deleteTable(tableName);
+    assertFalse(catalog.existsTable(tableName));
+  }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByColumn() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.COLUMN);
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(catalog.existsTable(tableName));
+    catalog.addTable(desc);
+    assertTrue(catalog.existsTable(tableName));
+
+    TableDesc retrieved = catalog.getTableDesc(tableName);
+
+    assertEquals(retrieved.getName(), tableName);
+    assertEquals(retrieved.getPartitions().getPartitionsType(), CatalogProtos.PartitionsType.COLUMN);
+    assertEquals(retrieved.getPartitions().getColumn(0).getColumnName(), "id");
+
+    catalog.deleteTable(tableName);
+    assertFalse(catalog.existsTable(tableName));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java
index 260b4c7..d3671b3 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java
@@ -22,6 +22,9 @@ 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.tajo.catalog.partition.Partitions;
+import org.apache.tajo.catalog.partition.Specifier;
+import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.catalog.store.AbstractDBStore;
@@ -219,4 +222,175 @@ public class TestDBStore {
           s2.getColumn(i).getColumnName());
     }
   }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByHash1() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.HASH);
+    partitions.setNumPartitions(2);
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(store.existTable(tableName));
+    store.addTable(desc);
+    assertTrue(store.existTable(tableName));
+
+    TableDesc retrieved = store.getTable(tableName);
+
+    // Schema order check
+    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
+    store.deleteTable(tableName);
+    assertFalse(store.existTable(tableName));
+  }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByHash2() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.HASH);
+    partitions.setNumPartitions(2);
+
+    partitions.addSpecifier(new Specifier("sub_part1"));
+    partitions.addSpecifier(new Specifier("sub_part2"));
+    partitions.addSpecifier(new Specifier("sub_part3"));
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(store.existTable(tableName));
+    store.addTable(desc);
+    assertTrue(store.existTable(tableName));
+
+    TableDesc retrieved = store.getTable(tableName);
+
+    // Schema order check
+    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
+    store.deleteTable(tableName);
+    assertFalse(store.existTable(tableName));
+  }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByList() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.LIST);
+
+    partitions.addSpecifier(new Specifier("sub_part1", "Seoul,서울"));
+    partitions.addSpecifier(new Specifier("sub_part2", "Busan,부산"));
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(store.existTable(tableName));
+    store.addTable(desc);
+    assertTrue(store.existTable(tableName));
+
+    TableDesc retrieved = store.getTable(tableName);
+
+    // Schema order check
+    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
+    store.deleteTable(tableName);
+    assertFalse(store.existTable(tableName));
+  }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByRange() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.RANGE);
+
+    partitions.addSpecifier(new Specifier("sub_part1", "2"));
+    partitions.addSpecifier(new Specifier("sub_part2", "5"));
+    partitions.addSpecifier(new Specifier("sub_part3"));
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(store.existTable(tableName));
+    store.addTable(desc);
+    assertTrue(store.existTable(tableName));
+
+    TableDesc retrieved = store.getTable(tableName);
+
+    // Schema order check
+    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
+    store.deleteTable(tableName);
+    assertFalse(store.existTable(tableName));
+  }
+
+  @Test
+  public final void testAddAndDeleteTablePartitionByColumn() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4)
+        .addColumn("name", Type.TEXT)
+        .addColumn("age", Type.INT4)
+        .addColumn("score", Type.FLOAT8);
+
+    String tableName = "addedtable";
+    Options opts = new Options();
+    opts.put("file.delimiter", ",");
+    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
+
+    Partitions partitions = new Partitions();
+    partitions.addColumn(new Column("id", Type.INT4));
+    partitions.setPartitionsType(CatalogProtos.PartitionsType.COLUMN);
+
+    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
+    desc.setPartitions(partitions);
+    assertFalse(store.existTable(tableName));
+    store.addTable(desc);
+    assertTrue(store.existTable(tableName));
+
+    TableDesc retrieved = store.getTable(tableName);
+
+    // Schema order check
+    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
+    store.deleteTable(tableName);
+    assertFalse(store.existTable(tableName));
+  }
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
index c94260f..58eabfd 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
@@ -27,8 +27,10 @@ import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoProtos.QueryState;
+import org.apache.tajo.algebra.CreateTable;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.partition.Specifier;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.client.QueryStatus;
 import org.apache.tajo.client.TajoClient;
@@ -463,6 +465,40 @@ public class TajoCli {
       }
       sb.append("\n");
     }
+
+    sb.append("\n");
+    sb.append("Partitions: \n");
+    if (desc.getPartitions() != null) {
+      sb.append("type:").append(desc.getPartitions().getPartitionsType().name()).append("\n");
+      if (desc.getPartitions().getNumPartitions() > 0)
+        sb.append("numbers:").append(desc.getPartitions().getNumPartitions()).append("\n");
+
+      sb.append("columns:").append("\n");
+      for(Column eachColumn: desc.getPartitions().getColumns()) {
+        sb.append("  ");
+        sb.append(eachColumn.getColumnName()).append("\t").append(eachColumn.getDataType().getType());
+        if (eachColumn.getDataType().hasLength()) {
+          sb.append("(").append(eachColumn.getDataType().getLength()).append(")");
+        }
+        sb.append("\n");
+      }
+
+      if (desc.getPartitions().getSpecifiers() != null) {
+        sb.append("specifier:").append("\n");
+        for(Specifier specifier :desc.getPartitions().getSpecifiers()) {
+          sb.append("  ");
+          sb.append("name:").append(specifier.getName());
+          if (!specifier.getExpressions().equals("")) {
+            sb.append(", expressions:").append(specifier.getExpressions());
+          } else {
+            if (desc.getPartitions().getPartitionsType().name().equals(CreateTable.PartitionType.RANGE))
+              sb.append(" expressions: MAXVALUE");
+          }
+          sb.append("\n");
+        }
+      }
+    }
+
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
index 4a305ae..5bba89b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
@@ -222,6 +222,18 @@ public class LogicalPlan {
         return ensureUniqueColumn(candidates);
       }
 
+      // Trying to find columns from schema in current block.
+      if (block.getSchema() != null) {
+        Column found = block.getSchema().getColumnByName(columnRef.getName());
+        if (found != null) {
+          candidates.add(found);
+        }
+      }
+
+      if (!candidates.isEmpty()) {
+        return ensureUniqueColumn(candidates);
+      }
+
       throw new VerifyException("ERROR: no such a column name "+ columnRef.getCanonicalName());
     }
   }
@@ -724,7 +736,6 @@ public class LogicalPlan {
         // add target to list if a target can be evaluated at this node
         List<Integer> newEvaluatedTargetIds = new ArrayList<Integer>();
         for (int i = 0; i < getTargetListNum(); i++) {
-
           if (getTarget(i) != null && !isTargetResolved(i)) {
             EvalNode expr = getTarget(i).getEvalTree();
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
index 79f02d1..b5f8d2f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.algebra.CreateTable.ColumnDefinition;
 import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.partition.Partitions;
+import org.apache.tajo.catalog.partition.Specifier;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.DataType;
@@ -46,6 +48,7 @@ import org.apache.tajo.engine.planner.LogicalPlan.QueryBlock;
 import org.apache.tajo.engine.planner.logical.*;
 import org.apache.tajo.engine.utils.SchemaUtil;
 import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.util.TUtil;
 
 import java.util.List;
 import java.util.Stack;
@@ -732,7 +735,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
   public LogicalNode visitCreateTable(PlanContext context, Stack<OpType> stack, CreateTable expr)
       throws PlanningException {
 
-     String tableName = expr.getTableName();
+    String tableName = expr.getTableName();
 
     if (expr.hasSubQuery()) {
       stack.add(OpType.CreateTable);
@@ -789,10 +792,125 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
         createTableNode.setPath(new Path(expr.getLocation()));
       }
 
+      if (expr.hasPartition()) {
+        createTableNode.setPartitions(convertTableElementsPartition(context, expr));
+      }
       return createTableNode;
     }
   }
 
+  /**
+   * convert table elements into Partition.
+   *
+   * @param context
+   * @param expr
+   * @return
+   * @throws PlanningException
+   */
+  private Partitions convertTableElementsPartition(PlanContext context,
+                                                   CreateTable expr) throws PlanningException {
+    Schema schema = convertTableElementsSchema(expr.getTableElements());
+    Partitions partitions = null;
+    List<Specifier> specifiers = null;
+    if (expr.hasPartition()) {
+      partitions = new Partitions();
+      specifiers = TUtil.newList();
+
+      partitions.setPartitionsType(CatalogProtos.PartitionsType.valueOf(expr.getPartition()
+          .getPartitionType().name()));
+
+      if (expr.getPartition().getPartitionType().equals(CreateTable.PartitionType.HASH)) {
+        CreateTable.HashPartition hashPartition = expr.getPartition();
+
+        partitions.setColumns(convertTableElementsColumns(expr.getTableElements()
+            , hashPartition.getColumns()));
+
+        if (hashPartition.getColumns() != null) {
+          if (hashPartition.getQuantifier() != null) {
+            String quantity = ((LiteralValue)hashPartition.getQuantifier()).getValue();
+            partitions.setNumPartitions(Integer.parseInt(quantity));
+          }
+
+          if (hashPartition.getSpecifiers() != null) {
+            for(CreateTable.PartitionSpecifier eachSpec: hashPartition.getSpecifiers()) {
+              specifiers.add(new Specifier(eachSpec.getName()));
+            }
+          }
+
+          if (specifiers.isEmpty() && partitions.getNumPartitions() > 0) {
+            for (int i = 0; i < partitions.getNumPartitions(); i++) {
+              String partitionName = partitions.getPartitionsType().name() + "_" + expr
+                  .getTableName() + "_" + i;
+              specifiers.add(new Specifier(partitionName));
+            }
+          }
+
+          if (!specifiers.isEmpty())
+            partitions.setSpecifiers(specifiers);
+        }
+      } else if (expr.getPartition().getPartitionType().equals(CreateTable.PartitionType.LIST)) {
+        CreateTable.ListPartition listPartition = expr.getPartition();
+
+        partitions.setColumns(convertTableElementsColumns(expr.getTableElements()
+            , listPartition.getColumns()));
+
+        if (listPartition.getSpecifiers() != null) {
+          StringBuffer sb = new StringBuffer();
+
+          for(CreateTable.ListPartitionSpecifier eachSpec: listPartition.getSpecifiers()) {
+            Specifier specifier = new Specifier(eachSpec.getName());
+            sb.delete(0, sb.length());
+            for(Expr eachExpr : eachSpec.getValueList().getValues()) {
+              context.block.setSchema(schema);
+              EvalNode eval = createEvalTree(context.plan, context.block, eachExpr);
+              if(sb.length() > 1)
+                sb.append(",");
+
+              sb.append(eval.toString());
+            }
+            specifier.setExpressions(sb.toString());
+            specifiers.add(specifier);
+          }
+          if (!specifiers.isEmpty())
+            partitions.setSpecifiers(specifiers);
+        }
+      } else if (expr.getPartition().getPartitionType().equals(CreateTable.PartitionType.RANGE)) {
+        CreateTable.RangePartition rangePartition = expr.getPartition();
+
+        partitions.setColumns(convertTableElementsColumns(expr.getTableElements()
+            , rangePartition.getColumns()));
+
+        if (rangePartition.getSpecifiers() != null) {
+          for(CreateTable.RangePartitionSpecifier eachSpec: rangePartition.getSpecifiers()) {
+            Specifier specifier = new Specifier();
+
+            if (eachSpec.getName() != null)
+              specifier.setName(eachSpec.getName());
+
+            if (eachSpec.getEnd() != null) {
+              context.block.setSchema(schema);
+              EvalNode eval = createEvalTree(context.plan, context.block, eachSpec.getEnd());
+              specifier.setExpressions(eval.toString());
+            }
+
+            if(eachSpec.isEndMaxValue()) {
+              specifier.setExpressions(null);
+            }
+            specifiers.add(specifier);
+          }
+          if (!specifiers.isEmpty())
+            partitions.setSpecifiers(specifiers);
+        }
+      } else if (expr.getPartition().getPartitionType().equals(CreateTable.PartitionType.COLUMN)) {
+        CreateTable.ColumnPartition columnPartition = expr.getPartition();
+
+        partitions.setColumns(convertTableElementsColumns(expr.getTableElements()
+            , columnPartition.getColumns()));
+      }
+    }
+
+    return partitions;
+  }
 
 
   /**
@@ -811,6 +929,21 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     return schema;
   }
 
+  private List<Column> convertTableElementsColumns(CreateTable.ColumnDefinition [] elements,
+                                                   ColumnReferenceExpr[] references) {
+    List<Column> columnList = TUtil.newList();
+
+    for(CreateTable.ColumnDefinition columnDefinition: elements) {
+      for(ColumnReferenceExpr eachReference: references) {
+        if (columnDefinition.getColumnName().equalsIgnoreCase(eachReference.getName())) {
+          columnList.add(convertColumn(columnDefinition));
+        }
+      }
+    }
+
+    return columnList;
+  }
+
   private DataType convertDataType(org.apache.tajo.algebra.DataType dataType) {
     TajoDataTypes.Type type = TajoDataTypes.Type.valueOf(dataType.getTypeName());
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
index 50656c5..942309d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Options;
 import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.partition.Partitions;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.engine.planner.PlanString;
 import org.apache.tajo.util.TUtil;
@@ -35,6 +36,7 @@ public class CreateTableNode extends LogicalNode implements Cloneable {
   @Expose private Path path;
   @Expose private Options options;
   @Expose private boolean external;
+  @Expose private Partitions partitions;
 
   public CreateTableNode(int pid, String tableName, Schema schema) {
     super(pid, NodeType.CREATE_TABLE);
@@ -90,6 +92,17 @@ public class CreateTableNode extends LogicalNode implements Cloneable {
     this.external = external;
   }
 
+  public Partitions getPartitions() {
+    return partitions;
+  }
+
+  public void setPartitions(Partitions partitions) {
+    this.partitions = partitions;
+  }
+
+  public boolean hasPartition() {
+    return this.partitions != null;
+  }
 
   @Override
   public PlanString getPlanString() {
@@ -107,7 +120,8 @@ public class CreateTableNode extends LogicalNode implements Cloneable {
           && this.external == other.external
           && TUtil.checkEquals(path, other.path)
           && TUtil.checkEquals(options, other.options)
-          && TUtil.checkEquals(partitionKeys, other.partitionKeys);
+          && TUtil.checkEquals(partitionKeys, other.partitionKeys)
+          && TUtil.checkEquals(partitions, other.partitions);
     } else {
       return false;
     }
@@ -123,6 +137,7 @@ public class CreateTableNode extends LogicalNode implements Cloneable {
     store.path = path != null ? new Path(path.toString()) : null;
     store.partitionKeys = partitionKeys != null ? partitionKeys.clone() : null;
     store.options = (Options) (options != null ? options.clone() : null);
+    store.partitions = (Partitions) (partitions != null ? partitions.clone() : null);
     return store;
   }
   
@@ -142,6 +157,7 @@ public class CreateTableNode extends LogicalNode implements Cloneable {
     sb.append(",\"storeType\": \"" + this.storageType);
     sb.append(",\"path\" : \"" + this.path).append("\",");
     sb.append(",\"external\" : \"" + this.external).append("\",");
+    sb.append(",\"partitions\" : \"" + this.partitions).append("\",");
     
     sb.append("\n  \"out schema\": ").append(getOutSchema()).append(",")
     .append("\n  \"in schema\": ").append(getInSchema())

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
index 94447c0..b2bd937 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.partition.Partitions;
 import org.apache.tajo.engine.planner.PlanString;
 import org.apache.tajo.util.TUtil;
 
@@ -38,12 +39,19 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
   @Expose private Options options;
   @Expose private boolean isCreatedTable = false;
   @Expose private boolean isOverwritten = false;
+  @Expose private Partitions partitions;
 
   public StoreTableNode(int pid, String tableName) {
     super(pid, NodeType.STORE);
     this.tableName = tableName;
   }
 
+  public StoreTableNode(int pid, String tableName, Partitions partitions) {
+    super(pid, NodeType.STORE);
+    this.tableName = tableName;
+    this.partitions = partitions;
+  }
+
   public final String getTableName() {
     return this.tableName;
   }
@@ -101,6 +109,13 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
     return this.options;
   }
 
+  public Partitions getPartitions() {
+    return partitions;
+  }
+
+  public void setPartitions(Partitions partitions) {
+    this.partitions = partitions;
+  }
 
   @Override
   public PlanString getPlanString() {
@@ -131,6 +146,7 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
       eq = eq && TUtil.checkEquals(options, other.options);
       eq = eq && isCreatedTable == other.isCreatedTable;
       eq = eq && isOverwritten == other.isOverwritten;
+      eq = eq && TUtil.checkEquals(partitions, other.partitions);
       return eq;
     } else {
       return false;
@@ -147,6 +163,7 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
     store.options = options != null ? (Options) options.clone() : null;
     store.isCreatedTable = isCreatedTable;
     store.isOverwritten = isOverwritten;
+    store.partitions = partitions;
     return store;
   }
 
@@ -169,8 +186,13 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
     }
     
     sb.append("\n  \"out schema\": ").append(getOutSchema()).append(",")
-    .append("\n  \"in schema\": ").append(getInSchema())
-    .append("}");
+    .append("\n  \"in schema\": ").append(getInSchema());
+
+    if(partitions != null) {
+      sb.append(partitions.toString());
+    }
+
+    sb.append("}");
     
     return sb.toString() + "\n"
         + getChild().toString();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 505fd71..4f18b11 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -32,6 +32,7 @@ import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.exception.AlreadyExistsTableException;
 import org.apache.tajo.catalog.exception.NoSuchTableException;
+import org.apache.tajo.catalog.partition.Partitions;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
@@ -131,7 +132,7 @@ public class GlobalEngine extends AbstractService {
       }
 
       Expr planningContext = hiveQueryMode ? converter.parse(sql) : analyzer.parse(sql);
-      
+
       LogicalPlan plan = createLogicalPlan(planningContext);
       LogicalRootNode rootNode = plan.getRootBlock().getRoot();
 
@@ -253,10 +254,11 @@ public class GlobalEngine extends AbstractService {
     }
 
     return createTableOnDirectory(createTable.getTableName(), createTable.getSchema(), meta,
-        createTable.getPath(), true);
+        createTable.getPath(), true, createTable.getPartitions());
   }
 
-  public TableDesc createTableOnDirectory(String tableName, Schema schema, TableMeta meta, Path path, boolean isCreated)
+  public TableDesc createTableOnDirectory(String tableName, Schema schema, TableMeta meta,
+                                          Path path, boolean isCreated, Partitions partitions)
       throws IOException {
     if (catalog.existsTable(tableName)) {
       throw new AlreadyExistsTableException(tableName);
@@ -284,6 +286,7 @@ public class GlobalEngine extends AbstractService {
     stats.setNumBytes(totalSize);
     TableDesc desc = CatalogUtil.newTableDesc(tableName, schema, meta, path);
     desc.setStats(stats);
+    desc.setPartitions(partitions);
     catalog.addTable(desc);
 
     LOG.info("Table " + desc.getName() + " is created (" + desc.getStats().getNumBytes() + ")");

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index d1faf4f..03cb4d0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -32,6 +32,7 @@ import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.exception.NoSuchTableException;
+import org.apache.tajo.catalog.partition.Partitions;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.ipc.ClientProtos;
@@ -307,11 +308,12 @@ public class TajoMasterClientService extends AbstractService {
 
         Schema schema = new Schema(request.getSchema());
         TableMeta meta = new TableMeta(request.getMeta());
+        Partitions partitions = new Partitions(request.getPartitions());
 
         TableDesc desc;
         try {
-          desc = context.getGlobalEngine().createTableOnDirectory(request.getName(), schema, meta, path,
-              false);
+          desc = context.getGlobalEngine().createTableOnDirectory(request.getName(), schema,
+              meta, path, false, partitions);
         } catch (Exception e) {
           return TableResponse.newBuilder()
               .setResultCode(ResultCode.ERROR)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
index 89c40c8..dc9c905 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
@@ -124,6 +124,7 @@ message CreateTableRequest {
   required SchemaProto schema = 2;
   required TableProto meta = 3;
   required string path = 4;
+  optional PartitionsProto partitions = 5;
 }
 
 message AttachTableRequest {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/0b0de13b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
index bd62f40..8b06ebf 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -191,4 +191,126 @@ public class TestTajoClient {
     assertEquals(tableName1, desc.getName());
     assertTrue(desc.getStats().getNumBytes() > 0);
   }
+
+  @Test
+  public final void testCreateAndDropTablePartitionedHash1ByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedHash1";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += " PARTITION BY HASH (deptname)";
+    sql += " (PARTITION sub_part1, PARTITION sub_part2, PARTITION sub_part3)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndDropTablePartitionedHash2ByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedHash2";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY HASH (deptname)";
+    sql += "PARTITIONS 2";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndDropTablePartitionedListByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedList";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY LIST (deptname)";
+    sql += "( PARTITION sub_part1 VALUES('r&d', 'design'),";
+    sql += "PARTITION sub_part2 VALUES('sales', 'hr') )";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
+  @Test
+  public final void testCreateAndDropTablePartitionedRangeByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedRange";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY RANGE (score)";
+    sql += "( PARTITION sub_part1 VALUES LESS THAN (2),";
+    sql += "PARTITION sub_part2 VALUES LESS THAN (5),";
+    sql += "PARTITION sub_part2 VALUES LESS THAN (MAXVALUE) )";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+  @Test
+  public final void testCreateAndDropTablePartitionedColumnByExecuteQuery() throws IOException,
+      ServiceException, SQLException {
+    TajoConf conf = cluster.getConfiguration();
+    final String tableName = "testCreateAndDropTablePartitionedColumn";
+
+    assertFalse(client.existTable(tableName));
+
+    String sql = "create table " + tableName + " (deptname text, score int4)";
+    sql += "PARTITION BY COLUMN (deptname)";
+
+    client.updateQuery(sql);
+    assertTrue(client.existTable(tableName));
+
+    Path tablePath = client.getTableDesc(tableName).getPath();
+    FileSystem hdfs = tablePath.getFileSystem(conf);
+    assertTrue(hdfs.exists(tablePath));
+
+    client.updateQuery("drop table " + tableName);
+    assertFalse(client.existTable(tableName));
+    assertFalse(hdfs.exists(tablePath));
+  }
+
 }


[06/18] git commit: TAJO-346: Implement hex function. (DaeMyung Kang via hyunsik)

Posted by ji...@apache.org.
TAJO-346: Implement hex function. (DaeMyung Kang via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: 778c01f8f71ea6dd4573e83c5b62ccefcd39b337
Parents: c268316
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 2 16:56:50 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 2 16:56:50 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../tajo/engine/function/string/ToHex.java      | 68 ++++++++++++++++++++
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  7 +-
 .../java/org/apache/tajo/master/TajoMaster.java |  8 +++
 .../TestStringOperatorsAndFunctions.java        | 15 +++++
 5 files changed, 99 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/778c01f8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6b97b85..caadc0c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-346: Implement hex function. (DaeMyung Kang via hyunsik)
+
     TAJO-349: Implement md5(text). (DaeMyung Kang via hyunsik)
 
     TAJO-351: Implement reverse(text). (DaeMyung Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/778c01f8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/ToHex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/ToHex.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/ToHex.java
new file mode 100644
index 0000000..c5bebda
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/ToHex.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.Float4Datum;
+import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+import org.apache.commons.codec.binary.Hex;
+import java.nio.ByteBuffer;
+
+/**
+ * Function definition
+ *
+ * text to_hex(text)
+ * text to_hex(int)
+ */
+public class ToHex extends GeneralFunction {
+
+  public ToHex() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  public String trimZero(String hexString) {
+    int len = hexString.length();
+    for (int i = 0; i < len; i++) {
+        if (hexString.charAt(i) != '0') {
+            return hexString.substring(i);
+        }
+    }
+
+    return hexString;
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String ret = new String(Hex.encodeHex(datum.asByteArray()));
+    return DatumFactory.createText(trimZero(ret));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/778c01f8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 2b641cb..48d5858 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -740,7 +740,12 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
   @Override
   public LiteralValue visitUnsigned_numeric_literal(@NotNull SQLParser.Unsigned_numeric_literalContext ctx) {
     if (ctx.NUMBER() != null) {
-      return new LiteralValue(ctx.getText(), LiteralType.Unsigned_Integer);
+      long lValue = Long.parseLong(ctx.getText());
+      if (lValue >= Integer.MIN_VALUE && lValue <= Integer.MAX_VALUE) {
+        return new LiteralValue(ctx.getText(), LiteralType.Unsigned_Integer);
+      } else {
+        return new LiteralValue(ctx.getText(), LiteralType.Unsigned_Large_Integer);
+      } 
     } else {
       return new LiteralValue(ctx.getText(), LiteralType.Unsigned_Float);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/778c01f8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 6be7c83..a3a85c4 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -345,6 +345,14 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
 
     sqlFuncs.add(
+        new FunctionDesc("to_hex", ToHex.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.INT4)));
+    sqlFuncs.add(
+        new FunctionDesc("to_hex", ToHex.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.INT8)));
+    sqlFuncs.add(
         new FunctionDesc("upper", Upper.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/778c01f8/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index 2e46101..221249a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -224,6 +224,21 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
   }
 
   @Test
+  public void testHex() throws IOException {
+    testSimpleEval("select to_hex(1) as col1 ", new String[]{"1"});
+    testSimpleEval("select to_hex(10) as col1 ", new String[]{"a"});
+    testSimpleEval("select to_hex(1234) as col1 ", new String[]{"4d2"});
+    testSimpleEval("select to_hex(1023456788888888) as col1 ", new String[]{"3a2d41a583d38"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", ",abcdef,3.14", "select to_hex(10) from table1",
+        new String[]{"a"});
+  }
+
+  @Test
   public void testSubstr() throws IOException {
     testSimpleEval("select substr('abcdef', 3, 2) as col1 ", new String[]{"cd"});
     testSimpleEval("select substr('abcdef', 3) as col1 ", new String[]{"cdef"});


[02/18] git commit: TAJO-350: Implement round, floor, ceil. (hyoungjunkim via hyunsik)

Posted by ji...@apache.org.
TAJO-350: Implement round, floor, ceil. (hyoungjunkim via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: 3233d3824b8b8c572afe096f41511794d2633e9c
Parents: dc56c8e
Author: Hyunsik Choi <hy...@apache.org>
Authored: Sun Dec 1 18:37:16 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Sun Dec 1 18:37:16 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/tajo/engine/function/math/Ceil.java  | 50 ++++++++++++
 .../apache/tajo/engine/function/math/Floor.java | 50 ++++++++++++
 .../apache/tajo/engine/function/math/Round.java | 50 ++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java | 31 ++++++++
 .../tajo/engine/function/TestMethFunction.java  | 83 ++++++++++++++++++++
 6 files changed, 266 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3233d382/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b99cc69..06bbc5c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-350: Implement round, floor, ceil. (hyoungjunkim via hyunsik)
+
     TAJO-207: Implement bit_length(string) function. (DaeMyung Kang via jihoon)
 
     TAJO-341: Implement substr function. (hyoungjunkim via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3233d382/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Ceil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Ceil.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Ceil.java
new file mode 100644
index 0000000..5a593bc
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Ceil.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 ceil(value FLOAT8)
+ */
+public class Ceil extends GeneralFunction {
+  public Ceil() {
+    super(new Column[] {
+      new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8((long)Math.ceil(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3233d382/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Floor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Floor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Floor.java
new file mode 100644
index 0000000..e48a733
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Floor.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 floor(value FLOAT8)
+ */
+public class Floor extends GeneralFunction {
+  public Floor() {
+    super(new Column[] {
+      new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8((long)Math.floor(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3233d382/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Round.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Round.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Round.java
new file mode 100644
index 0000000..f95e330
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Round.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 round(value FLOAT8)
+ */
+public class Round extends GeneralFunction {
+  public Round() {
+    super(new Column[] {
+      new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createInt8(Math.round(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3233d382/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 1c823d3..9071da1 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -46,6 +46,9 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.function.Country;
 import org.apache.tajo.engine.function.InCountry;
 import org.apache.tajo.engine.function.builtin.*;
+import org.apache.tajo.engine.function.math.Ceil;
+import org.apache.tajo.engine.function.math.Floor;
+import org.apache.tajo.engine.function.math.Round;
 import org.apache.tajo.engine.function.string.*;
 import org.apache.tajo.master.querymaster.QueryJobManager;
 import org.apache.tajo.master.rm.TajoWorkerResourceManager;
@@ -407,6 +410,34 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.INT4, Type.INT4)));
 
+    sqlFuncs.add(
+        new FunctionDesc("round", Round.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+    sqlFuncs.add(
+        new FunctionDesc("round", Round.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
+    sqlFuncs.add(
+        new FunctionDesc("floor", Floor.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+    sqlFuncs.add(
+        new FunctionDesc("floor", Floor.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
+    sqlFuncs.add(
+        new FunctionDesc("ceil", Ceil.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("ceil", Ceil.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
     return sqlFuncs;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3233d382/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java
new file mode 100644
index 0000000..77b0443
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMethFunction.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.FLOAT8;
+
+public class TestMethFunction extends ExprTestBase {
+  @Test
+  public void testRound() throws IOException {
+    testSimpleEval("select round(5.1) as col1 ", new String[]{"5"});
+    testSimpleEval("select round(5.5) as col1 ", new String[]{"6"});
+    testSimpleEval("select round(5.6) as col1 ", new String[]{"6"});
+
+//    testSimpleEval("select round(-5.1) as col1 ", new String[]{"-5"});
+//    testSimpleEval("select round(-5.5) as col1 ", new String[]{"-6"});
+//    testSimpleEval("select round(-5.6) as col1 ", new String[]{"-6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.4", "select round(col1 + col2 + col3) from table1",
+        new String[]{"2"});
+  }
+
+  @Test
+  public void testFloor() throws IOException {
+    testSimpleEval("select floor(5.1) as col1 ", new String[]{"5"});
+    testSimpleEval("select floor(5.5) as col1 ", new String[]{"5"});
+    testSimpleEval("select floor(5.6) as col1 ", new String[]{"5"});
+//    testSimpleEval("select floor(-5.1) as col1 ", new String[]{"-6"});
+//    testSimpleEval("select floor(-5.6) as col1 ", new String[]{"-6"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.4", "select floor(col1 + col2 + col3) from table1",
+        new String[]{"1"});
+  }
+
+  @Test
+  public void testCeil() throws IOException {
+    testSimpleEval("select ceil(5.0) as col1 ", new String[]{"5"});
+    testSimpleEval("select ceil(5.1) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceil(5.5) as col1 ", new String[]{"6"});
+    testSimpleEval("select ceil(5.6) as col1 ", new String[]{"6"});
+//    testSimpleEval("select ceil(-5.1) as col1 ", new String[]{"-5"});
+//    testSimpleEval("select ceil(-5.6) as col1 ", new String[]{"-5"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from table1",
+        new String[]{"2"});
+  }
+}


[05/18] git commit: TAJO-349: Implement md5(text). (DaeMyung Kang via hyunsik)

Posted by ji...@apache.org.
TAJO-349: Implement md5(text). (DaeMyung Kang via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: c26831669afd25b4e34fe7fad457eab4e98bcb00
Parents: 14ec0ec
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 2 14:30:59 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 2 14:30:59 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/tajo/engine/function/string/Md5.java | 55 ++++++++++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  6 +++
 .../TestStringOperatorsAndFunctions.java        | 12 +++++
 4 files changed, 75 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c2683166/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f051e1d..6b97b85 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-349: Implement md5(text). (DaeMyung Kang via hyunsik)
+
     TAJO-351: Implement reverse(text). (DaeMyung Kang via hyunsik)
 
     TAJO-342: Implement strpos(string, substring) function. 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c2683166/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Md5.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Md5.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Md5.java
new file mode 100644
index 0000000..fc1db8d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Md5.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+import java.security.*;
+import org.apache.commons.codec.binary.Hex;
+
+/**
+ * Function definition
+ *
+ * text md5(string text)
+ */
+public class Md5 extends GeneralFunction {
+  public Md5() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    try {
+        MessageDigest md = MessageDigest.getInstance("MD5");
+        return DatumFactory.createText(new String(Hex.encodeHex(md.digest(datum.asByteArray()))));
+    } catch (NoSuchAlgorithmException e){
+        return NullDatum.get();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c2683166/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 557fd5a..6be7c83 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -352,6 +352,12 @@ public class TajoMaster extends CompositeService {
         new FunctionDesc("lower", Lower.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
+
+    sqlFuncs.add(
+        new FunctionDesc("md5", Md5.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
+
     sqlFuncs.add(
         new FunctionDesc("char_length", CharLength.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c2683166/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index b62b1b8..2e46101 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -212,6 +212,18 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
   }
 
   @Test
+  public void testMd5() throws IOException {
+    testSimpleEval("select md5('1') as col1 ", new String[]{"c4ca4238a0b923820dcc509a6f75849b"});
+    testSimpleEval("select md5('tajo') as col1 ", new String[]{"742721b3a79f71a9491681b8e8a7ce85"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "abc,efg,3.14", "select md5(col1) from table1", new String[]{"900150983cd24fb0d6963f7d28e17f72"});
+  }
+
+  @Test
   public void testSubstr() throws IOException {
     testSimpleEval("select substr('abcdef', 3, 2) as col1 ", new String[]{"cd"});
     testSimpleEval("select substr('abcdef', 3) as col1 ", new String[]{"cdef"});


[17/18] git commit: TAJO-366: Implement trigonometric functions. (Jae Young Lee via jihoon)

Posted by ji...@apache.org.
TAJO-366: Implement trigonometric functions. (Jae Young Lee via jihoon)


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

Branch: refs/heads/DAG-execplan
Commit: abc08308f7fe54619c56c6dbba5e4c0e7b669aa8
Parents: 3e2a263
Author: Jihoon Son <ji...@apache.org>
Authored: Tue Dec 3 19:47:43 2013 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Tue Dec 3 19:47:43 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../apache/tajo/engine/function/math/Acos.java  |  50 +++++++++
 .../apache/tajo/engine/function/math/Asin.java  |  50 +++++++++
 .../apache/tajo/engine/function/math/Atan.java  |  50 +++++++++
 .../apache/tajo/engine/function/math/Atan2.java |  51 +++++++++
 .../apache/tajo/engine/function/math/Cos.java   |  50 +++++++++
 .../apache/tajo/engine/function/math/Sin.java   |  28 ++---
 .../apache/tajo/engine/function/math/Tan.java   |  51 +++++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  62 +++++++++++
 .../tajo/engine/function/TestMathFunctions.java | 103 ++++++++++++++++++-
 10 files changed, 481 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8dd57d6..be5418c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-366: Implement trigonometric functions. (Jae Young Lee via jihoon)
+
     TAJO-358: Implement initcap(string) function. (Seungun Choe via hyunsik)
 
     TAJO-355: Implement repeat(text,int) function. (DaeMyung Kang via jaehwa)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Acos.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Acos.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Acos.java
new file mode 100644
index 0000000..3c12bfb
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Acos.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 acos(value FLOAT8)
+ */
+public class Acos extends GeneralFunction {
+  public Acos() {
+    super(new Column[] {
+        new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.acos(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Asin.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Asin.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Asin.java
new file mode 100644
index 0000000..9357a1a
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Asin.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 asin(value FLOAT8)
+ */
+public class Asin extends GeneralFunction {
+  public Asin() {
+    super(new Column[] {
+        new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.asin(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan.java
new file mode 100644
index 0000000..c628a26
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 atan(value FLOAT8)
+ */
+public class Atan extends GeneralFunction {
+  public Atan() {
+    super(new Column[] {
+        new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.atan(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan2.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan2.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan2.java
new file mode 100644
index 0000000..99908e7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Atan2.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 atan2(value FLOAT8, value FLOAT8)
+ */
+public class Atan2 extends GeneralFunction {
+  public Atan2() {
+    super(new Column[] {
+        new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatumx = params.get(0);
+    Datum valueDatumy = params.get(1);
+    if(valueDatumx instanceof NullDatum || valueDatumy instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.atan2(valueDatumx.asFloat8(), valueDatumy.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Cos.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Cos.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Cos.java
new file mode 100644
index 0000000..44194b2
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Cos.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 cos(value FLOAT8)
+ */
+public class Cos extends GeneralFunction {
+  public Cos() {
+    super(new Column[] {
+        new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.cos(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java
index 158fad2..928625c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java
@@ -29,22 +29,22 @@ import org.apache.tajo.storage.Tuple;
 /**
  * Function definition
  *
- * INT8 sin(value FLOAT8)
+ * Float8 sin(value FLOAT8)
  */
 public class Sin extends GeneralFunction {
-    public Sin() {
-        super(new Column[] {
-                new Column("value", TajoDataTypes.Type.FLOAT8)
-        });
-    }
-
-    @Override
-    public Datum eval(Tuple params) {
-        Datum valueDatum = params.get(0);
-        if(valueDatum instanceof NullDatum) {
-            return NullDatum.get();
-        }
+  public Sin() {
+    super(new Column[] {
+        new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
 
-        return DatumFactory.createFloat8(Math.sin(valueDatum.asFloat8()));
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
     }
+
+    return DatumFactory.createFloat8(Math.sin(valueDatum.asFloat8()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Tan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Tan.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Tan.java
new file mode 100644
index 0000000..d06461b
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Tan.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * Float8 tan(value FLOAT8)
+ */
+
+public class Tan extends GeneralFunction {
+  public Tan() {
+    super(new Column[] {
+        new Column("value", TajoDataTypes.Type.FLOAT8)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.tan(valueDatum.asFloat8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 37ace1f..f783a0f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -494,6 +494,68 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
 
     sqlFuncs.add(
+        new FunctionDesc("cos", Cos.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("cos", Cos.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
+    sqlFuncs.add(
+        new FunctionDesc("tan", Tan.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("tan", Tan.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
+    sqlFuncs.add(
+        new FunctionDesc("asin", Asin.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("asin", Asin.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
+    sqlFuncs.add(
+        new FunctionDesc("acos", Acos.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("acos", Acos.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
+    sqlFuncs.add(
+        new FunctionDesc("atan", Atan.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("atan", Atan.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
+    sqlFuncs.add(
+        new FunctionDesc("atan2", Atan2.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8, Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("atan2", Atan2.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4, Type.FLOAT4)));
+
+
+
+    sqlFuncs.add(
         new FunctionDesc("initcap", InitCap.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/abc08308/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
index 2c95043..703d49d 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
@@ -94,7 +94,106 @@ public class TestMathFunctions extends ExprTestBase {
     schema.addColumn("col3", FLOAT8);
 
     testEval(schema, "table1", "1.0, 0.2, 0.1", "select sin(col1 + col2 + col3) from table1",
-       new String[]{"0.963558185417193"});
-}
+        new String[]{"0.963558185417193"});
+  }
+
+
+  @Test
+  public void testCos() throws IOException {
+    testSimpleEval("select cos(0.0) as col1 ", new String[]{"1.0"});
+    testSimpleEval("select cos(0.7) as col1 ", new String[]{"0.7648421949641616"});
+    testSimpleEval("select cos(1.2) as col1 ", new String[]{"0.36235771003358624"});
+//    testSimpleEval("select cos(-0.5) as col1 ", new String[]{"0.8775825618903728"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select cos(col1 + col2 + col3) from table1",
+        new String[]{"0.26749882862458735"});
+  }
+
+  @Test
+  public void testTan() throws IOException {
+    testSimpleEval("select tan(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select tan(0.3) as col1 ", new String[]{"0.30933626267125297"});
+    testSimpleEval("select tan(0.8) as col1 ", new String[]{"1.0296385816093323"});
+//    testSimpleEval("select tan(-0.5) as col1 ", new String[]{"-0.5463024898437905"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select tan(col1 - col2 - col3) from table1",
+        new String[]{"0.8422883804630795"});
+  }
+
+  @Test
+  public void testAsin() throws IOException {
+    testSimpleEval("select asin(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select asin(0.3) as col1 ", new String[]{"0.3046926665119266"});
+    testSimpleEval("select asin(0.8) as col1 ", new String[]{"0.9272952378698274"});
+//    testSimpleEval("select asin(-0.5) as col1 ", new String[]{"-0.5235987755982989"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select asin(col1 - col2 - col3) from table1",
+        new String[]{"0.7753974966107532"});
+  }
+
+  @Test
+  public void testAcos() throws IOException {
+    testSimpleEval("select acos(0.0) as col1 ", new String[]{"1.5707963267948966"});
+    testSimpleEval("select acos(0.3) as col1 ", new String[]{"1.26610366028297"});
+    testSimpleEval("select acos(0.8) as col1 ", new String[]{"0.6435010889250692"});
+//    testSimpleEval("select acos(-0.5) as col1 ", new String[]{"2.0943951023931957"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select acos(col1 - col2 - col3) from table1",
+        new String[]{"0.7953988301841435"});
+  }
+
+  @Test
+  public void testAtan() throws IOException {
+    testSimpleEval("select atan(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select atan(0.8) as col1 ", new String[]{"0.6747409494924117"});
+    testSimpleEval("select atan(1.2) as col1 ", new String[]{"0.8760580701406995"});
+//    testSimpleEval("select atan(-0.5) as col1 ", new String[]{"-0.4636476090008061"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan(col1 + col2 + col3) from table1",
+        new String[]{"0.9151007005533605"});
+  }
+
+  @Test
+  public void testAtan2() throws IOException {
+    testSimpleEval("select atan2(0.8, 0.0) as col1 ", new String[]{"1.5707963267948966"});
+    testSimpleEval("select atan2(0.8, 1.1) as col1 ", new String[]{"0.6287962831935603"});
+    testSimpleEval("select atan2(2.7, 0.3) as col1 ", new String[]{"1.460139103198048"});
+//    testSimpleEval("select atan(-0.5, 0.3) as col1 ", new String[]{"-1.0303768265243125"});
+//    testSimpleEval("select atan(-0.2, -1.3) as col1 ", new String[]{"-2.988943325194528"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select atan2(col1 + col2, col3) from table1",
+        new String[]{"1.4876550949064553"});
+  }
+
 
 }


[09/18] git commit: TAJO-352: Implement right/left(text, size) function. (DaeMyung Kang via hyunsik)

Posted by ji...@apache.org.
TAJO-352: Implement right/left(text, size) function. (DaeMyung Kang via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: 43ec04aaa34ffe5c5da2cfd05ff919ffac44044b
Parents: 528c914
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 2 17:23:50 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 2 17:24:29 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  6 +-
 .../java/org/apache/tajo/datum/TextDatum.java   |  2 +
 .../org/apache/tajo/engine/parser/SQLParser.g4  | 12 +++-
 .../tajo/engine/function/string/Left.java       | 73 +++++++++++++++++++
 .../tajo/engine/function/string/Right.java      | 74 ++++++++++++++++++++
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  2 +-
 .../java/org/apache/tajo/master/TajoMaster.java |  8 +++
 .../TestStringOperatorsAndFunctions.java        | 56 +++++++++++++++
 8 files changed, 230 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1646d4c..0ad8c3c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,9 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-352: Implement right/left(text, size) function. 
+    (DaeMyung Kang via hyunsik)
+
     TAJO-346: Implement hex function. (DaeMyung Kang via hyunsik)
 
     TAJO-349: Implement md5(text). (DaeMyung Kang via hyunsik)
@@ -34,7 +37,8 @@ Release 0.8.0 - unreleased
 
   IMPROVEMENTS
 
-    TAJO-317: Improve TajoResourceManager to support more elaborate resource management. (Keuntae Park via jihoon)
+    TAJO-317: Improve TajoResourceManager to support more elaborate resource 
+    management. (Keuntae Park via jihoon)
 
     TAJO-314: Make TaskScheduler be pluggable. (jihoon)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
index 779d5ea..a7234b7 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
@@ -30,6 +30,8 @@ public class TextDatum extends Datum {
   @Expose private int size;
   @Expose private byte [] bytes;
 
+  public static final TextDatum EMPTY_TEXT = new TextDatum("");
+
   public TextDatum() {
     super(TajoDataTypes.Type.TEXT);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
index b071b82..2cc415f 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
@@ -1050,7 +1050,17 @@ unique_predicate
 */
 
 routine_invocation
-  : Identifier LEFT_PAREN sql_argument_list? RIGHT_PAREN
+  : function_name LEFT_PAREN sql_argument_list? RIGHT_PAREN
+  ;
+
+function_names_for_reserved_words
+  : LEFT
+  | RIGHT
+  ;
+
+function_name
+  : Identifier
+  | function_names_for_reserved_words
   ;
 
 sql_argument_list

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Left.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Left.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Left.java
new file mode 100644
index 0000000..3d25cd8
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Left.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.engine.function.string;
+
+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.TextDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text left(string text, int size)
+ */
+public class Left extends GeneralFunction {
+  public Left() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("size", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  public int getSize(int length, int size) {
+    if (size < 0) {
+        size = length + size;
+        if (size < 0) {
+            size = 0;
+        }
+    }
+
+    return (size < length) ? size : length;
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    Datum sizeDatum = params.get(1);
+    if(sizeDatum instanceof NullDatum) return NullDatum.get();
+
+    String data = datum.asChars();
+    int length = data.length();
+    int size = sizeDatum.asInt4();
+
+    size = getSize(length, size);
+    if (size == 0) {
+        return TextDatum.EMPTY_TEXT;
+    }
+
+    return DatumFactory.createText(data.substring(0, size));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Right.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Right.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Right.java
new file mode 100644
index 0000000..43421fd
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Right.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.TextDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text right(string text, int size)
+ */
+public class Right extends GeneralFunction {
+  public Right() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("size", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  public int getSize(int length, int size) {
+    if (size < 0) {
+        size = length + size;
+        if (size < 0) {
+            size = 0;
+        }
+    }
+
+    return (size < length) ? size : length;
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    Datum sizeDatum = params.get(1);
+    if(sizeDatum instanceof NullDatum) return NullDatum.get();
+
+    String data = datum.asChars();
+    int length = data.length();
+    int size = sizeDatum.asInt4();
+
+    size = getSize(length, size);
+    if (size == 0) {
+        return TextDatum.EMPTY_TEXT;
+    }
+
+    int startIdx = length - size;
+    return DatumFactory.createText(data.substring(startIdx, length));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 48d5858..04980f7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -769,7 +769,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
 
   @Override
   public FunctionExpr visitRoutine_invocation(SQLParser.Routine_invocationContext ctx) {
-    String signature = ctx.Identifier().getText();
+    String signature = ctx.function_name().getText();
     FunctionExpr function = new FunctionExpr(signature);
     if (ctx.sql_argument_list() != null) {
       int numArgs = ctx.sql_argument_list().value_expression().size();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index a3a85c4..2fe4213 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -345,6 +345,14 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
 
     sqlFuncs.add(
+        new FunctionDesc("left", Left.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.INT4)));
+    sqlFuncs.add(
+        new FunctionDesc("right", Right.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.INT4)));
+    sqlFuncs.add(
         new FunctionDesc("to_hex", ToHex.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.INT4)));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/43ec04aa/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index 221249a..c62c494 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -150,6 +150,62 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
   }
 
   @Test
+  public void testLeft() throws IOException {
+    testSimpleEval("select left('abcdef',1) as col1 ", new String[]{"a"});
+    testSimpleEval("select left('abcdef',2) as col1 ", new String[]{"ab"});
+    testSimpleEval("select left('abcdef',3) as col1 ", new String[]{"abc"});
+    testSimpleEval("select left('abcdef',4) as col1 ", new String[]{"abcd"});
+    testSimpleEval("select left('abcdef',5) as col1 ", new String[]{"abcde"});
+    testSimpleEval("select left('abcdef',6) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select left('abcdef',7) as col1 ", new String[]{"abcdef"});
+//    testSimpleEval("select from_left('abcdef',-1) as col1 ", new String[]{"abcde"});
+//    testSimpleEval("select from_left('abcdef',-2) as col1 ", new String[]{"abcd"});
+//    testSimpleEval("select from_left('abcdef',-3) as col1 ", new String[]{"abc"});
+//    testSimpleEval("select from_left('abcdef',-4) as col1 ", new String[]{"ab"});
+//    testSimpleEval("select from_left('abcdef',-5) as col1 ", new String[]{"a"});
+//    testSimpleEval("select from_left('abcdef',-6) as col1 ", new String[]{""});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", INT4);
+    schema.addColumn("col3", TEXT);
+
+    // for null tests
+    testEval(schema, "table1", ",1,ghi", "select left(col1,1) is null from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc,,ghi", "select left(col1,col2) is null from table1", new String[]{"t"});
+
+    testEval(schema, "table1", "abc,1,ghi", "select left(col1,1) || left(col3,3) from table1", new String[]{"aghi"});
+  }
+
+  @Test
+  public void testRight() throws IOException {
+    testSimpleEval("select right('abcdef',1) as col1 ", new String[]{"f"});
+    testSimpleEval("select right('abcdef',2) as col1 ", new String[]{"ef"});
+    testSimpleEval("select right('abcdef',3) as col1 ", new String[]{"def"});
+    testSimpleEval("select right('abcdef',4) as col1 ", new String[]{"cdef"});
+    testSimpleEval("select right('abcdef',5) as col1 ", new String[]{"bcdef"});
+    testSimpleEval("select right('abcdef',6) as col1 ", new String[]{"abcdef"});
+    testSimpleEval("select right('abcdef',7) as col1 ", new String[]{"abcdef"});
+//    testSimpleEval("select from_right('abcdef',-1) as col1 ", new String[]{"bcdef"});
+//    testSimpleEval("select from_right('abcdef',-2) as col1 ", new String[]{"cdef"});
+//    testSimpleEval("select from_right('abcdef',-3) as col1 ", new String[]{"def"});
+//    testSimpleEval("select from_right('abcdef',-4) as col1 ", new String[]{"ef"});
+//    testSimpleEval("select from_right('abcdef',-5) as col1 ", new String[]{"f"});
+//    testSimpleEval("select from_right('abcdef',-6) as col1 ", new String[]{""});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", INT4);
+    schema.addColumn("col3", TEXT);
+
+    // for null tests
+    testEval(schema, "table1", ",1,ghi", "select right(col1,1) is null from table1", new String[]{"t"});
+    testEval(schema, "table1", "abc,,ghi", "select right(col1,col2) is null from table1", new String[]{"t"});
+
+    testEval(schema, "table1", "abc,1,ghi", "select right(col1,1) || right(col3,3) from table1", new String[]{"cghi"});
+  }
+
+  @Test
   public void testReverse() throws IOException {
     testSimpleEval("select reverse('abcdef') as col1 ", new String[]{"fedcba"});
     testSimpleEval("select reverse('가') as col1 ", new String[]{"가"});


[03/18] git commit: TAJO-342: Implement strpos(string, substring) function. (hyoungjunkim via hyunsik)

Posted by ji...@apache.org.
TAJO-342: Implement strpos(string, substring) function. (hyoungjunkim via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: e29b062d69315d24d75d121b53987e24e0451071
Parents: 3233d38
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 2 13:22:43 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 2 13:22:43 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../tajo/engine/function/string/StrPos.java     |  62 +++++++++++
 .../tajo/engine/function/string/StrPosb.java    | 109 +++++++++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  10 ++
 .../TestStringOperatorsAndFunctions.java        |  32 ++++++
 5 files changed, 216 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e29b062d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 06bbc5c..ebb1af0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,9 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-342: Implement strpos(string, substring) function. 
+    (hyoungjunkim via hyunsik)
+
     TAJO-350: Implement round, floor, ceil. (hyoungjunkim via hyunsik)
 
     TAJO-207: Implement bit_length(string) function. (DaeMyung Kang via jihoon)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e29b062d/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPos.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPos.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPos.java
new file mode 100644
index 0000000..46fdded
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPos.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * int strpos(string text, substring text))
+ */
+public class StrPos extends GeneralFunction {
+  public StrPos() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("substring", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    Datum substringDatum = params.get(1);
+    if(substringDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String value = valueDatum.asChars();
+    String substring = substringDatum.asChars();
+    if(substring.length() == 0) {
+      return DatumFactory.createInt4(1);
+    }
+
+    return DatumFactory.createInt4(value.indexOf(substring) + 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e29b062d/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java
new file mode 100644
index 0000000..511d092
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/StrPosb.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Function definition
+ *
+ * int strposb(string text, substring text))
+ */
+public class StrPosb extends GeneralFunction {
+  public StrPosb() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("substring", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    if(valueDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    Datum substringDatum = params.get(1);
+    if(substringDatum instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    String value = valueDatum.asChars();
+    String substring = substringDatum.asChars();
+    if(substring.length() == 0) {
+      return DatumFactory.createInt4(1);
+    }
+
+    return DatumFactory.createInt4(findText(value, substring) + 1);
+  }
+
+  /**
+   * finds the location of specified substring.
+   * @param value
+   * @param substring
+   * @return
+   */
+  public static int findText(String value, String substring) {
+    //This method is copied from Hive's GenericUDFUtils.findText()
+    int start = 0;
+    byte[] valueBytes = value.getBytes();
+    byte[] substrBytes = substring.getBytes();
+
+    ByteBuffer src = ByteBuffer.wrap(valueBytes, 0, valueBytes.length);
+    ByteBuffer tgt = ByteBuffer.wrap(substrBytes, 0, substrBytes.length);
+    byte b = tgt.get();
+    src.position(start);
+
+    while (src.hasRemaining()) {
+      if (b == src.get()) { // matching first byte
+        src.mark(); // save position in loop
+        tgt.mark(); // save position in target
+        boolean found = true;
+        int pos = src.position() - 1;
+        while (tgt.hasRemaining()) {
+          if (!src.hasRemaining()) { // src expired first
+            tgt.reset();
+            src.reset();
+            found = false;
+            break;
+          }
+          if (!(tgt.get() == src.get())) {
+            tgt.reset();
+            src.reset();
+            found = false;
+            break; // no match
+          }
+        }
+        if (found) {
+          return pos;
+        }
+      }
+    }
+    return -1; // not found
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e29b062d/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 9071da1..5b9c00e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -438,6 +438,16 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataType(Type.INT8),
             CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
 
+    sqlFuncs.add(
+        new FunctionDesc("strpos", StrPos.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT4),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.TEXT)));
+
+    sqlFuncs.add(
+        new FunctionDesc("strposb", StrPosb.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.INT4),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.TEXT)));
+
     return sqlFuncs;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e29b062d/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index f9da0c4..042cc6f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -242,4 +242,36 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
     testEval(schema, "table1", "ABC,DEF,3.14", "select bit_length(lower(col1) || lower(col2)) from table1",
         new String[]{"48"});
   }
+
+  @Test
+  public void testStrpos() throws IOException {
+    testSimpleEval("select strpos('tajo','jo') as col1 ", new String[]{"3"});
+    testSimpleEval("select strpos('tajo','') as col1 ", new String[]{"1"});
+    testSimpleEval("select strpos('tajo','abcdef') as col1 ", new String[]{"0"});
+    testSimpleEval("select strpos('일이삼사오육','삼사') as col1 ", new String[]{"3"});
+    testSimpleEval("select strpos('일이삼사오육','일이삼') as col1 ", new String[]{"1"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strpos(lower(col1) || lower(col2), 'fh') from table1",
+        new String[]{"6"});
+  }
+
+  @Test
+  public void testStrposb() throws IOException {
+    testSimpleEval("select strposb('tajo','jo') as col1 ", new String[]{"3"});
+    testSimpleEval("select strposb('tajo','') as col1 ", new String[]{"1"});
+    testSimpleEval("select strposb('tajo','abcdef') as col1 ", new String[]{"0"});
+    testSimpleEval("select strposb('일이삼사오육','삼사') as col1 ", new String[]{"7"});    //utf8 1 korean word = 3 chars
+    testSimpleEval("select strposb('일이삼사오육','삼사일') as col1 ", new String[]{"0"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strposb(lower(col1) || lower(col2), 'fh') from table1",
+        new String[]{"6"});
+  }
 }


[13/18] git commit: TAJO-355: Implement repeat(text, int) function. (DaeMyung Kang via jaehwa)

Posted by ji...@apache.org.
TAJO-355: Implement repeat(text,int) function. (DaeMyung Kang via jaehwa)


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

Branch: refs/heads/DAG-execplan
Commit: 29a0aa01c5e68422841170b86537eab169eabf26
Parents: 2659cda
Author: blrunner <jh...@gruter.com>
Authored: Tue Dec 3 16:18:57 2013 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Tue Dec 3 16:18:57 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../tajo/engine/function/string/Repeat.java     | 61 ++++++++++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  5 ++
 .../TestStringOperatorsAndFunctions.java        | 14 +++++
 4 files changed, 82 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/29a0aa01/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 257dd93..ab0e893 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-355: Implement repeat(text,int) function. (DaeMyung Kang via jaehwa)
+
     TAJO-339: Implement sin( x ) - returns the sine of x (x is in radians). (Jae Young Lee via jaehwa)
 
     TAJO-348: Implement octet_length(text). (DaeMyung Kang via jaehwa)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/29a0aa01/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Repeat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Repeat.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Repeat.java
new file mode 100644
index 0000000..c170e5d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/Repeat.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text reverse(string text)
+ */
+public class Repeat extends GeneralFunction {
+  public Repeat() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT),
+        new Column("count", TajoDataTypes.Type.INT4)
+    });
+  }
+
+  private String repeat(String word, int count) {
+    StringBuilder builder = new StringBuilder();
+    for (int i = 0; i < count; i++) {
+        builder.append(word);
+    } 
+
+    return builder.toString();
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    Datum countDatum = params.get(1);
+    if(countDatum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(repeat(datum.asChars(), countDatum.asInt4()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/29a0aa01/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index d3e9cae..58c3ef1 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -343,6 +343,11 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
 
     sqlFuncs.add(
+        new FunctionDesc("repeat", Repeat.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.INT4)));
+
+    sqlFuncs.add(
         new FunctionDesc("left", Left.class, FunctionType.GENERAL,
             CatalogUtil.newSimpleDataType(Type.TEXT),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.INT4)));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/29a0aa01/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index bbe6939..24c658e 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -218,6 +218,20 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
   }
 
   @Test
+  public void testRepeat() throws IOException {
+    testSimpleEval("select repeat('ab',4) as col1 ", new String[]{"abababab"});
+    testSimpleEval("select repeat('가',3) as col1 ", new String[]{"가가가"});
+    testSimpleEval("select repeat('a',2) as col1 ", new String[]{"aa"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+    schema.addColumn("col3", TEXT);
+    testEval(schema, "table1", "abc,efg,3.14", "select repeat(col1,2) from table1", new String[]{"abcabc"});
+  }
+
+
+  @Test
   public void testUpper() throws IOException {
     testSimpleEval("select upper('abcdef') as col1 ", new String[]{"ABCDEF"});
 


[16/18] git commit: TAJO-358: Implement initcap(string) function. (Seungun Choe via hyunsik)

Posted by ji...@apache.org.
TAJO-358: Implement initcap(string) function. (Seungun Choe via hyunsik)


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

Branch: refs/heads/DAG-execplan
Commit: 3e2a2636e902137740d5e6e3510af565d8d33106
Parents: 0b0de13
Author: Hyunsik Choi <hy...@apache.org>
Authored: Tue Dec 3 18:41:41 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Tue Dec 3 18:41:41 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  5 +-
 .../tajo/engine/function/string/InitCap.java    | 49 ++++++++++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  5 ++
 .../TestStringOperatorsAndFunctions.java        |  6 +++
 4 files changed, 64 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3e2a2636/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8c53d9b..8dd57d6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,9 +4,12 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-358: Implement initcap(string) function. (Seungun Choe via hyunsik)
+
     TAJO-355: Implement repeat(text,int) function. (DaeMyung Kang via jaehwa)
 
-    TAJO-339: Implement sin( x ) - returns the sine of x (x is in radians). (Jae Young Lee via jaehwa)
+    TAJO-339: Implement sin( x ) - returns the sine of x (x is in radians). 
+    (Jae Young Lee via jaehwa)
 
     TAJO-348: Implement octet_length(text). (DaeMyung Kang via jaehwa)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3e2a2636/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/InitCap.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/InitCap.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/InitCap.java
new file mode 100644
index 0000000..e2158fb
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/string/InitCap.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.string;
+
+import org.apache.commons.lang.WordUtils;
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * text initcap(string text)
+ */
+public class InitCap extends GeneralFunction {
+  public InitCap() {
+    super(new Column[] {
+        new Column("text", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum datum = params.get(0);
+    if(datum instanceof NullDatum) return NullDatum.get();
+
+    return DatumFactory.createText(WordUtils.capitalizeFully(datum.asChars()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3e2a2636/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 58c3ef1..37ace1f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -493,6 +493,11 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataType(Type.FLOAT8),
             CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
 
+    sqlFuncs.add(
+        new FunctionDesc("initcap", InitCap.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.TEXT)));
+
     return sqlFuncs;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/3e2a2636/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index 24c658e..b65fd34 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -397,4 +397,10 @@ public class TestStringOperatorsAndFunctions extends ExprTestBase {
     testEval(schema, "table1", "ABCDEF,HIJKLMN,3.14", "select strposb(lower(col1) || lower(col2), 'fh') from table1",
         new String[]{"6"});
   }
+
+  @Test
+  public void testInitcap() throws IOException {
+    testSimpleEval("select initcap('hi bro') ", new String[]{"Hi Bro"});
+    testSimpleEval("select initcap('HI BRO') ", new String[]{"Hi Bro"});
+  }
 }


[12/18] git commit: TAJO-339: Implement sin( x ) - returns the sine of x (x is in radians). (Jae Young Lee via jaehwa)

Posted by ji...@apache.org.
TAJO-339: Implement sin( x ) - returns the sine of x (x is in radians). (Jae Young Lee via jaehwa)


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

Branch: refs/heads/DAG-execplan
Commit: 2659cda4a88d074a7664babc39ac6ec1719778eb
Parents: 19c7585
Author: blrunner <jh...@gruter.com>
Authored: Tue Dec 3 16:11:55 2013 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Tue Dec 3 16:11:55 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/tajo/engine/function/math/Sin.java   | 50 ++++++++++++++++++++
 .../java/org/apache/tajo/master/TajoMaster.java | 10 ++++
 .../tajo/engine/function/TestMathFunctions.java | 17 +++++++
 4 files changed, 79 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/2659cda4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5967f4b..257dd93 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-339: Implement sin( x ) - returns the sine of x (x is in radians). (Jae Young Lee via jaehwa)
+
     TAJO-348: Implement octet_length(text). (DaeMyung Kang via jaehwa)
 
     TAJO-357: Fix invalid filename TestMethFunction to TestMathFUnction.

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/2659cda4/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java
new file mode 100644
index 0000000..158fad2
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/math/Sin.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.math;
+
+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.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+/**
+ * Function definition
+ *
+ * INT8 sin(value FLOAT8)
+ */
+public class Sin extends GeneralFunction {
+    public Sin() {
+        super(new Column[] {
+                new Column("value", TajoDataTypes.Type.FLOAT8)
+        });
+    }
+
+    @Override
+    public Datum eval(Tuple params) {
+        Datum valueDatum = params.get(0);
+        if(valueDatum instanceof NullDatum) {
+            return NullDatum.get();
+        }
+
+        return DatumFactory.createFloat8(Math.sin(valueDatum.asFloat8()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/2659cda4/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 8139b8e..d3e9cae 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -478,6 +478,16 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataType(Type.INT4),
             CatalogUtil.newSimpleDataTypeArray(Type.TEXT, Type.TEXT)));
 
+    sqlFuncs.add(
+        new FunctionDesc("sin", Sin.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
+
+    sqlFuncs.add(
+        new FunctionDesc("sin", Sin.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.FLOAT8),
+            CatalogUtil.newSimpleDataTypeArray(Type.FLOAT4)));
+
     return sqlFuncs;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/2659cda4/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
index 6a59b7a..2c95043 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
@@ -80,4 +80,21 @@ public class TestMathFunctions extends ExprTestBase {
     testEval(schema, "table1", "1.0, 0.2, 0.1", "select ceil(col1 + col2 + col3) from table1",
         new String[]{"2"});
   }
+
+  @Test
+  public void testSin() throws IOException {
+    testSimpleEval("select sin(0.0) as col1 ", new String[]{"0.0"});
+    testSimpleEval("select sin(0.7) as col1 ", new String[]{"0.6442176781200616"});
+    testSimpleEval("select sin(1.2) as col1 ", new String[]{"0.9320391032457895"});
+//    testSimpleEval("select sin(-0.5) as col1 ", new String[]{"-0.479425538604203"});
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", FLOAT8);
+    schema.addColumn("col2", FLOAT8);
+    schema.addColumn("col3", FLOAT8);
+
+    testEval(schema, "table1", "1.0, 0.2, 0.1", "select sin(col1 + col2 + col3) from table1",
+       new String[]{"0.963558185417193"});
+}
+
 }