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

[49/51] [partial] TAJO-22: The package prefix should be org.apache.tajo. (DaeMyung Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-algebra/src/main/java/tajo/algebra/TableSubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/tajo/algebra/TableSubQuery.java b/tajo-algebra/src/main/java/tajo/algebra/TableSubQuery.java
deleted file mode 100644
index e1881ae..0000000
--- a/tajo-algebra/src/main/java/tajo/algebra/TableSubQuery.java
+++ /dev/null
@@ -1,41 +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 tajo.algebra;
-
-public class TableSubQuery extends Relation {
-  private Expr subquery;
-
-  public TableSubQuery(String relName, Expr subquery) {
-    super(ExprType.TableSubQuery, relName);
-    this.subquery = subquery;
-  }
-
-  public Expr getSubQuery() {
-    return subquery;
-  }
-
-  @Override
-  boolean equalsTo(Expr expr) {
-    return subquery.equals(subquery);
-  }
-
-  public String toJson() {
-    return JsonHelper.toJson(this);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-algebra/src/main/java/tajo/algebra/Target.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/tajo/algebra/Target.java b/tajo-algebra/src/main/java/tajo/algebra/Target.java
deleted file mode 100644
index ad79484..0000000
--- a/tajo-algebra/src/main/java/tajo/algebra/Target.java
+++ /dev/null
@@ -1,61 +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 tajo.algebra;
-
-import tajo.util.TUtil;
-
-public class Target implements JsonSerializable {
-  private Expr expr;
-  private String alias;
-
-  public Target(Expr expr) {
-   this.expr = expr;
-  }
-
-  public Expr getExpr() {
-    return expr;
-  }
-
-  public boolean hasAlias() {
-    return this.alias != null;
-  }
-
-  public String getAlias() {
-    return this.alias;
-  }
-
-  public void setAlias(String alias) {
-    this.alias = alias;
-  }
-
-  @Override
-  public String toJson() {
-    return JsonHelper.toJson(this);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof Target) {
-      Target another = (Target) obj;
-      return expr.equals(another.expr) && TUtil.checkEquals(alias, another.alias);
-    }
-
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-algebra/src/main/java/tajo/algebra/UnaryOperator.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/tajo/algebra/UnaryOperator.java b/tajo-algebra/src/main/java/tajo/algebra/UnaryOperator.java
deleted file mode 100644
index 9c38964..0000000
--- a/tajo-algebra/src/main/java/tajo/algebra/UnaryOperator.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.algebra;
-
-public abstract class UnaryOperator extends Expr {
-  private Expr child;
-
-  public UnaryOperator(ExprType type) {
-    super(type);
-  }
-
-  public Expr getChild() {
-    return this.child;
-  }
-
-  public void setChild(Expr op) {
-    this.child = op;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-algebra/src/test/java/org/apache/tajo/algebra/AlgebraTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/test/java/org/apache/tajo/algebra/AlgebraTestingUtil.java b/tajo-algebra/src/test/java/org/apache/tajo/algebra/AlgebraTestingUtil.java
new file mode 100644
index 0000000..51b8681
--- /dev/null
+++ b/tajo-algebra/src/test/java/org/apache/tajo/algebra/AlgebraTestingUtil.java
@@ -0,0 +1,31 @@
+/**
+ * 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.algebra;
+
+import static junit.framework.Assert.assertEquals;
+
+public class AlgebraTestingUtil {
+  public static JsonSerializable testJsonSerializer(JsonSerializable obj) {
+    Class clazz = obj.getClass();
+    String json = obj.toJson();
+    JsonSerializable restored = (JsonSerializable) JsonHelper.fromJson(json, clazz);
+    assertEquals(json, restored.toJson());
+    return restored;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-algebra/src/test/java/org/apache/tajo/algebra/TestExpr.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/test/java/org/apache/tajo/algebra/TestExpr.java b/tajo-algebra/src/test/java/org/apache/tajo/algebra/TestExpr.java
new file mode 100644
index 0000000..984c30f
--- /dev/null
+++ b/tajo-algebra/src/test/java/org/apache/tajo/algebra/TestExpr.java
@@ -0,0 +1,142 @@
+/**
+ * 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.algebra;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertFalse;
+import static org.apache.tajo.algebra.LiteralExpr.LiteralType;
+
+public class TestExpr {
+
+  @Test
+  public void testBinaryEquals() {
+    Expr expr1 = new BinaryOperator(ExprType.Plus,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    Expr expr2 = new BinaryOperator(ExprType.Plus,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    assertEquals(expr1, expr2);
+
+    Expr expr3 = new BinaryOperator(ExprType.Minus,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    assertFalse(expr1.equals(expr3));
+
+
+    Expr expr4 = new BinaryOperator(ExprType.Plus,
+        new LiteralExpr("2", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    assertFalse(expr1.equals(expr4));
+
+    Expr expr5 = new BinaryOperator(ExprType.Plus,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("3", LiteralType.Unsigned_Integer));
+
+    assertFalse(expr1.equals(expr5));
+  }
+
+  @Test
+  public void testBinaryHierarchyEquals() {
+    Expr left1 = new BinaryOperator(ExprType.Plus,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    Expr right1 = new BinaryOperator(ExprType.Multiply,
+        new LiteralExpr("2", LiteralType.Unsigned_Integer),
+        new LiteralExpr("3", LiteralType.Unsigned_Integer));
+
+    Expr one = new BinaryOperator(ExprType.Plus, left1, right1);
+
+    Expr left2 = new BinaryOperator(ExprType.Plus,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    Expr right2 = new BinaryOperator(ExprType.Multiply,
+        new LiteralExpr("2", LiteralType.Unsigned_Integer),
+        new LiteralExpr("3", LiteralType.Unsigned_Integer));
+
+    Expr two = new BinaryOperator(ExprType.Plus, left2, right2);
+
+    assertEquals(one, two);
+
+    Expr left3 = new BinaryOperator(ExprType.Minus,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    Expr wrong1 = new BinaryOperator(ExprType.Plus, left3, right1);
+
+    assertFalse(one.equals(wrong1));
+  }
+
+  @Test
+  public void testEquals() throws Exception {
+    Expr expr = new BinaryOperator(ExprType.LessThan,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    Relation relation = new Relation("employee");
+    Selection selection = new Selection(relation, expr);
+
+    Aggregation aggregation = new Aggregation();
+    aggregation.setTargets(new Target[]{
+          new Target(new ColumnReferenceExpr("col1"))
+        }
+    );
+
+    aggregation.setChild(selection);
+
+    Sort.SortSpec spec = new Sort.SortSpec(new ColumnReferenceExpr("col2"));
+    Sort sort = new Sort(new Sort.SortSpec[]{spec});
+    sort.setChild(aggregation);
+
+    assertEquals(sort, sort);
+
+    Expr different = generateOneExpr();
+    assertFalse(sort.equals(different));
+  }
+
+  private Expr generateOneExpr() {
+    Expr expr = new BinaryOperator(ExprType.LessThan,
+        new LiteralExpr("1", LiteralType.Unsigned_Integer),
+        new LiteralExpr("2", LiteralType.Unsigned_Integer));
+
+    Relation relation = new Relation("employee");
+    Selection selection = new Selection(relation, expr);
+
+    Aggregation aggregation = new Aggregation();
+    aggregation.setTargets(new Target[]{
+        new Target(new ColumnReferenceExpr("col1"))
+    });
+
+    aggregation.setChild(selection);
+
+    Sort.SortSpec spec = new Sort.SortSpec(new ColumnReferenceExpr("col2"), false, false);
+    Sort sort = new Sort(new Sort.SortSpec[]{spec});
+    sort.setChild(aggregation);
+
+    return sort;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-algebra/src/test/java/tajo/algebra/AlgebraTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/test/java/tajo/algebra/AlgebraTestingUtil.java b/tajo-algebra/src/test/java/tajo/algebra/AlgebraTestingUtil.java
deleted file mode 100644
index 697e79d..0000000
--- a/tajo-algebra/src/test/java/tajo/algebra/AlgebraTestingUtil.java
+++ /dev/null
@@ -1,31 +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 tajo.algebra;
-
-import static junit.framework.Assert.assertEquals;
-
-public class AlgebraTestingUtil {
-  public static JsonSerializable testJsonSerializer(JsonSerializable obj) {
-    Class clazz = obj.getClass();
-    String json = obj.toJson();
-    JsonSerializable restored = (JsonSerializable) JsonHelper.fromJson(json, clazz);
-    assertEquals(json, restored.toJson());
-    return restored;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-algebra/src/test/java/tajo/algebra/TestExpr.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/test/java/tajo/algebra/TestExpr.java b/tajo-algebra/src/test/java/tajo/algebra/TestExpr.java
deleted file mode 100644
index d44c2ca..0000000
--- a/tajo-algebra/src/test/java/tajo/algebra/TestExpr.java
+++ /dev/null
@@ -1,142 +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 tajo.algebra;
-
-import org.junit.Test;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertFalse;
-import static tajo.algebra.LiteralExpr.LiteralType;
-
-public class TestExpr {
-
-  @Test
-  public void testBinaryEquals() {
-    Expr expr1 = new BinaryOperator(ExprType.Plus,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    Expr expr2 = new BinaryOperator(ExprType.Plus,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    assertEquals(expr1, expr2);
-
-    Expr expr3 = new BinaryOperator(ExprType.Minus,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    assertFalse(expr1.equals(expr3));
-
-
-    Expr expr4 = new BinaryOperator(ExprType.Plus,
-        new LiteralExpr("2", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    assertFalse(expr1.equals(expr4));
-
-    Expr expr5 = new BinaryOperator(ExprType.Plus,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("3", LiteralType.Unsigned_Integer));
-
-    assertFalse(expr1.equals(expr5));
-  }
-
-  @Test
-  public void testBinaryHierarchyEquals() {
-    Expr left1 = new BinaryOperator(ExprType.Plus,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    Expr right1 = new BinaryOperator(ExprType.Multiply,
-        new LiteralExpr("2", LiteralType.Unsigned_Integer),
-        new LiteralExpr("3", LiteralType.Unsigned_Integer));
-
-    Expr one = new BinaryOperator(ExprType.Plus, left1, right1);
-
-    Expr left2 = new BinaryOperator(ExprType.Plus,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    Expr right2 = new BinaryOperator(ExprType.Multiply,
-        new LiteralExpr("2", LiteralType.Unsigned_Integer),
-        new LiteralExpr("3", LiteralType.Unsigned_Integer));
-
-    Expr two = new BinaryOperator(ExprType.Plus, left2, right2);
-
-    assertEquals(one, two);
-
-    Expr left3 = new BinaryOperator(ExprType.Minus,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    Expr wrong1 = new BinaryOperator(ExprType.Plus, left3, right1);
-
-    assertFalse(one.equals(wrong1));
-  }
-
-  @Test
-  public void testEquals() throws Exception {
-    Expr expr = new BinaryOperator(ExprType.LessThan,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    Relation relation = new Relation("employee");
-    Selection selection = new Selection(relation, expr);
-
-    Aggregation aggregation = new Aggregation();
-    aggregation.setTargets(new Target[]{
-          new Target(new ColumnReferenceExpr("col1"))
-        }
-    );
-
-    aggregation.setChild(selection);
-
-    Sort.SortSpec spec = new Sort.SortSpec(new ColumnReferenceExpr("col2"));
-    Sort sort = new Sort(new Sort.SortSpec[]{spec});
-    sort.setChild(aggregation);
-
-    assertEquals(sort, sort);
-
-    Expr different = generateOneExpr();
-    assertFalse(sort.equals(different));
-  }
-
-  private Expr generateOneExpr() {
-    Expr expr = new BinaryOperator(ExprType.LessThan,
-        new LiteralExpr("1", LiteralType.Unsigned_Integer),
-        new LiteralExpr("2", LiteralType.Unsigned_Integer));
-
-    Relation relation = new Relation("employee");
-    Selection selection = new Selection(relation, expr);
-
-    Aggregation aggregation = new Aggregation();
-    aggregation.setTargets(new Target[]{
-        new Target(new ColumnReferenceExpr("col1"))
-    });
-
-    aggregation.setChild(selection);
-
-    Sort.SortSpec spec = new Sort.SortSpec(new ColumnReferenceExpr("col2"), false, false);
-    Sort sort = new Sort(new Sort.SortSpec[]{spec});
-    sort.setChild(aggregation);
-
-    return sort;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
new file mode 100644
index 0000000..f9eebfc
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
@@ -0,0 +1,261 @@
+/**
+ * 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;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.CatalogProtocol.CatalogProtocolService;
+import org.apache.tajo.catalog.proto.CatalogProtos.*;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * CatalogClient provides a client API to access the catalog server.
+ */
+public abstract class AbstractCatalogClient implements CatalogService {
+  private final Log LOG = LogFactory.getLog(AbstractCatalogClient.class);
+  protected CatalogProtocolService.BlockingInterface stub;
+
+  protected void setStub(CatalogProtocolService.BlockingInterface stub) {
+    this.stub = stub;
+  }
+
+  protected CatalogProtocolService.BlockingInterface getStub() {
+    return this.stub;
+  }
+
+  @Override
+  public final TableDesc getTableDesc(final String name) {
+    try {
+      return CatalogUtil.newTableDesc(stub.getTableDesc(null, StringProto.newBuilder()
+          .setValue(name).build()));
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return null;
+    }
+  }
+
+  @Override
+  public final Collection<String> getAllTableNames() {
+    List<String> protos = new ArrayList<String>();
+    GetAllTableNamesResponse response;
+
+    try {
+      response = stub.getAllTableNames(null, NullProto.newBuilder().build());
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return null;
+    }
+    int size = response.getTableNameCount();
+    for (int i = 0; i < size; i++) {
+      protos.add(response.getTableName(i));
+    }
+    return protos;
+  }
+
+  @Override
+  public final Collection<FunctionDesc> getFunctions() {
+    List<FunctionDesc> list = new ArrayList<FunctionDesc>();
+    GetFunctionsResponse response;
+    try {
+      response = stub.getFunctions(null, NullProto.newBuilder().build());
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return null;
+    }
+    int size = response.getFunctionDescCount();
+    for (int i = 0; i < size; i++) {
+      list.add(new FunctionDesc(response.getFunctionDesc(i)));
+    }
+    return list;
+  }
+
+  @Override
+  public final boolean addTable(final TableDesc desc) {
+    try {
+      return stub.addTable(null, (TableDescProto) desc.getProto()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final boolean deleteTable(final String name) {
+    try {
+      return stub.deleteTable(null,
+          StringProto.newBuilder().setValue(name).build()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final boolean existsTable(final String tableId) {
+    try {
+      return stub
+          .existsTable(null, StringProto.newBuilder().setValue(tableId).build())
+          .getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final boolean addIndex(IndexDesc index) {
+    try {
+      return stub.addIndex(null, index.getProto()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final boolean existIndex(String indexName) {
+    try {
+      return stub.existIndexByName(null, StringProto.newBuilder().
+          setValue(indexName).build()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public boolean existIndex(String tableName, String columnName) {
+    GetIndexRequest.Builder builder = GetIndexRequest.newBuilder();
+    builder.setTableName(tableName);
+    builder.setColumnName(columnName);
+    try {
+      return stub.existIndex(null, builder.build()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final IndexDesc getIndex(String indexName) {
+    try {
+      return new IndexDesc(
+          stub.getIndexByName(null,
+              StringProto.newBuilder().setValue(indexName).build()));
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return null;
+    }
+  }
+
+  @Override
+  public final IndexDesc getIndex(String tableName, String columnName) {
+    GetIndexRequest.Builder builder = GetIndexRequest.newBuilder();
+    builder.setTableName(tableName);
+    builder.setColumnName(columnName);
+    try {
+      return new IndexDesc(stub.getIndex(null, builder.build()));
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return null;
+    }
+  }
+
+  @Override
+  public boolean deleteIndex(String indexName) {
+    try {
+      return stub.delIndex(null,
+          StringProto.newBuilder().setValue(indexName).build()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final boolean registerFunction(final FunctionDesc funcDesc) {
+    try {
+      return stub.registerFunction(null, funcDesc.getProto()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final boolean unregisterFunction(final String signature,
+                                          DataType... paramTypes) {
+    UnregisterFunctionRequest.Builder builder =
+        UnregisterFunctionRequest.newBuilder();
+    builder.setSignature(signature);
+    for (DataType type : paramTypes) {
+      builder.addParameterTypes(type);
+    }
+    try {
+      return stub.unregisterFunction(null, builder.build()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+
+  @Override
+  public final FunctionDesc getFunction(final String signature,
+                                        DataType... paramTypes) {
+    GetFunctionMetaRequest.Builder builder =
+        GetFunctionMetaRequest.newBuilder();
+    builder.setSignature(signature);
+    for (DataType type : paramTypes) {
+      builder.addParameterTypes(type);
+    }
+
+    FunctionDescProto descProto;
+    try {
+      descProto = stub.getFunctionMeta(null, builder.build());
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return null;
+    }
+    return new FunctionDesc(descProto);
+  }
+
+  @Override
+  public final boolean containFunction(final String signature,
+                                       DataType... paramTypes) {
+    ContainFunctionRequest.Builder builder =
+        ContainFunctionRequest.newBuilder();
+    builder.setSignature(signature);
+    for (DataType type : paramTypes) {
+      builder.addParameterTypes(type);
+    }
+    try {
+      return stub.containFunction(null, builder.build()).getValue();
+    } catch (ServiceException e) {
+      LOG.error(e);
+      return false;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogClient.java
new file mode 100644
index 0000000..260106f
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogClient.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.catalog;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tajo.catalog.CatalogProtocol.CatalogProtocolService.BlockingInterface;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.rpc.ProtoBlockingRpcClient;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+/**
+ * CatalogClient provides a client API to access the catalog server.
+ */
+public class CatalogClient extends AbstractCatalogClient {
+  private final Log LOG = LogFactory.getLog(CatalogClient.class);
+  private final TajoConf conf;
+  private ProtoBlockingRpcClient client;
+
+  /**
+   * @throws java.io.IOException
+   *
+   */
+  public CatalogClient(final TajoConf conf) throws IOException {
+    this.conf = conf;
+    connect();
+  }
+
+  private void connect() throws IOException {
+    String serverName = conf.getVar(ConfVars.CATALOG_ADDRESS);
+    LOG.info("Trying to connect the catalog (" + serverName + ")");
+    InetSocketAddress addr = NetUtils.createSocketAddr(serverName);
+    try {
+      client = new ProtoBlockingRpcClient(
+          CatalogProtocol.class,
+          addr);
+      setStub((BlockingInterface) client.getStub());
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+
+    LOG.info("Connected to the catalog server (" + serverName + ")");
+  }
+
+  public void close() {
+    client.close();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java
new file mode 100644
index 0000000..c4eb535
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.catalog;
+
+import org.apache.tajo.common.TajoDataTypes.DataType;
+
+import java.util.Collection;
+
+public interface CatalogService {
+
+  /**
+   * Get a table description by name
+   * @param name table name
+   * @return a table description
+   * @see TableDescImpl
+   * @throws Throwable
+   */
+  TableDesc getTableDesc(String name);
+
+  /**
+   *
+   * @return
+   * @throws org.apache.tajo.catalog.exception.CatalogException
+   */
+  Collection<String> getAllTableNames();
+
+  /**
+   *
+   * @return
+   * @throws org.apache.tajo.catalog.exception.CatalogException
+   */
+  Collection<FunctionDesc> getFunctions();
+
+  /**
+   * Add a table via table description
+   * @see TableDescImpl
+   * @throws Throwable
+   */
+  boolean addTable(TableDesc desc);
+
+  /**
+   * Drop a table by name
+   *
+   * @param name table name
+   * @throws Throwable
+   */
+  boolean deleteTable(String name);
+
+  boolean existsTable(String tableId);
+
+  boolean addIndex(IndexDesc index);
+
+  boolean existIndex(String indexName);
+
+  boolean existIndex(String tableName, String columnName);
+
+  IndexDesc getIndex(String indexName);
+
+  IndexDesc getIndex(String tableName, String columnName);
+
+  boolean deleteIndex(String indexName);
+
+  boolean registerFunction(FunctionDesc funcDesc);
+
+  boolean unregisterFunction(String signature, DataType... paramTypes);
+
+  /**
+   *
+   * @param signature
+   * @return
+   */
+  FunctionDesc getFunction(String signature, DataType... paramTypes);
+
+  /**
+   *
+   * @param signature
+   * @return
+   */
+  boolean containFunction(String signature, DataType... paramTypes);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java
deleted file mode 100644
index f9dd78c..0000000
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java
+++ /dev/null
@@ -1,261 +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 tajo.catalog;
-
-import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import tajo.catalog.CatalogProtocol.CatalogProtocolService;
-import tajo.catalog.proto.CatalogProtos.*;
-import tajo.common.TajoDataTypes.DataType;
-import tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
-import tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * CatalogClient provides a client API to access the catalog server.
- */
-public abstract class AbstractCatalogClient implements CatalogService {
-  private final Log LOG = LogFactory.getLog(AbstractCatalogClient.class);
-  protected CatalogProtocolService.BlockingInterface stub;
-
-  protected void setStub(CatalogProtocolService.BlockingInterface stub) {
-    this.stub = stub;
-  }
-
-  protected CatalogProtocolService.BlockingInterface getStub() {
-    return this.stub;
-  }
-
-  @Override
-  public final TableDesc getTableDesc(final String name) {
-    try {
-      return CatalogUtil.newTableDesc(stub.getTableDesc(null, StringProto.newBuilder()
-          .setValue(name).build()));
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return null;
-    }
-  }
-
-  @Override
-  public final Collection<String> getAllTableNames() {
-    List<String> protos = new ArrayList<String>();
-    GetAllTableNamesResponse response;
-
-    try {
-      response = stub.getAllTableNames(null, NullProto.newBuilder().build());
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return null;
-    }
-    int size = response.getTableNameCount();
-    for (int i = 0; i < size; i++) {
-      protos.add(response.getTableName(i));
-    }
-    return protos;
-  }
-
-  @Override
-  public final Collection<FunctionDesc> getFunctions() {
-    List<FunctionDesc> list = new ArrayList<FunctionDesc>();
-    GetFunctionsResponse response;
-    try {
-      response = stub.getFunctions(null, NullProto.newBuilder().build());
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return null;
-    }
-    int size = response.getFunctionDescCount();
-    for (int i = 0; i < size; i++) {
-      list.add(new FunctionDesc(response.getFunctionDesc(i)));
-    }
-    return list;
-  }
-
-  @Override
-  public final boolean addTable(final TableDesc desc) {
-    try {
-      return stub.addTable(null, (TableDescProto) desc.getProto()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final boolean deleteTable(final String name) {
-    try {
-      return stub.deleteTable(null,
-          StringProto.newBuilder().setValue(name).build()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final boolean existsTable(final String tableId) {
-    try {
-      return stub
-          .existsTable(null, StringProto.newBuilder().setValue(tableId).build())
-          .getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final boolean addIndex(IndexDesc index) {
-    try {
-      return stub.addIndex(null, index.getProto()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final boolean existIndex(String indexName) {
-    try {
-      return stub.existIndexByName(null, StringProto.newBuilder().
-          setValue(indexName).build()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public boolean existIndex(String tableName, String columnName) {
-    GetIndexRequest.Builder builder = GetIndexRequest.newBuilder();
-    builder.setTableName(tableName);
-    builder.setColumnName(columnName);
-    try {
-      return stub.existIndex(null, builder.build()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final IndexDesc getIndex(String indexName) {
-    try {
-      return new IndexDesc(
-          stub.getIndexByName(null,
-              StringProto.newBuilder().setValue(indexName).build()));
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return null;
-    }
-  }
-
-  @Override
-  public final IndexDesc getIndex(String tableName, String columnName) {
-    GetIndexRequest.Builder builder = GetIndexRequest.newBuilder();
-    builder.setTableName(tableName);
-    builder.setColumnName(columnName);
-    try {
-      return new IndexDesc(stub.getIndex(null, builder.build()));
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return null;
-    }
-  }
-
-  @Override
-  public boolean deleteIndex(String indexName) {
-    try {
-      return stub.delIndex(null,
-          StringProto.newBuilder().setValue(indexName).build()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final boolean registerFunction(final FunctionDesc funcDesc) {
-    try {
-      return stub.registerFunction(null, funcDesc.getProto()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final boolean unregisterFunction(final String signature,
-                                          DataType... paramTypes) {
-    UnregisterFunctionRequest.Builder builder =
-        UnregisterFunctionRequest.newBuilder();
-    builder.setSignature(signature);
-    for (DataType type : paramTypes) {
-      builder.addParameterTypes(type);
-    }
-    try {
-      return stub.unregisterFunction(null, builder.build()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-
-  @Override
-  public final FunctionDesc getFunction(final String signature,
-                                        DataType... paramTypes) {
-    GetFunctionMetaRequest.Builder builder =
-        GetFunctionMetaRequest.newBuilder();
-    builder.setSignature(signature);
-    for (DataType type : paramTypes) {
-      builder.addParameterTypes(type);
-    }
-
-    FunctionDescProto descProto;
-    try {
-      descProto = stub.getFunctionMeta(null, builder.build());
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return null;
-    }
-    return new FunctionDesc(descProto);
-  }
-
-  @Override
-  public final boolean containFunction(final String signature,
-                                       DataType... paramTypes) {
-    ContainFunctionRequest.Builder builder =
-        ContainFunctionRequest.newBuilder();
-    builder.setSignature(signature);
-    for (DataType type : paramTypes) {
-      builder.addParameterTypes(type);
-    }
-    try {
-      return stub.containFunction(null, builder.build()).getValue();
-    } catch (ServiceException e) {
-      LOG.error(e);
-      return false;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogClient.java
deleted file mode 100644
index 6b2f183..0000000
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogClient.java
+++ /dev/null
@@ -1,68 +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 tajo.catalog;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.net.NetUtils;
-import tajo.catalog.CatalogProtocol.CatalogProtocolService.BlockingInterface;
-import tajo.conf.TajoConf;
-import tajo.conf.TajoConf.ConfVars;
-import tajo.rpc.ProtoBlockingRpcClient;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-/**
- * CatalogClient provides a client API to access the catalog server.
- */
-public class CatalogClient extends AbstractCatalogClient {
-  private final Log LOG = LogFactory.getLog(CatalogClient.class);
-  private final TajoConf conf;
-  private ProtoBlockingRpcClient client;
-
-  /**
-   * @throws java.io.IOException
-   *
-   */
-  public CatalogClient(final TajoConf conf) throws IOException {
-    this.conf = conf;
-    connect();
-  }
-
-  private void connect() throws IOException {
-    String serverName = conf.getVar(ConfVars.CATALOG_ADDRESS);
-    LOG.info("Trying to connect the catalog (" + serverName + ")");
-    InetSocketAddress addr = NetUtils.createSocketAddr(serverName);
-    try {
-      client = new ProtoBlockingRpcClient(
-          CatalogProtocol.class,
-          addr);
-      setStub((BlockingInterface) client.getStub());
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-
-    LOG.info("Connected to the catalog server (" + serverName + ")");
-  }
-
-  public void close() {
-    client.close();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java
deleted file mode 100644
index 823ade1..0000000
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java
+++ /dev/null
@@ -1,96 +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 tajo.catalog;
-
-import tajo.common.TajoDataTypes.DataType;
-
-import java.util.Collection;
-
-public interface CatalogService {
-
-  /**
-   * Get a table description by name
-   * @param name table name
-   * @return a table description
-   * @see TableDescImpl
-   * @throws Throwable
-   */
-  TableDesc getTableDesc(String name);
-
-  /**
-   *
-   * @return
-   * @throws tajo.catalog.exception.CatalogException
-   */
-  Collection<String> getAllTableNames();
-
-  /**
-   *
-   * @return
-   * @throws tajo.catalog.exception.CatalogException
-   */
-  Collection<FunctionDesc> getFunctions();
-
-  /**
-   * Add a table via table description
-   * @see TableDescImpl
-   * @throws Throwable
-   */
-  boolean addTable(TableDesc desc);
-
-  /**
-   * Drop a table by name
-   *
-   * @param name table name
-   * @throws Throwable
-   */
-  boolean deleteTable(String name);
-
-  boolean existsTable(String tableId);
-
-  boolean addIndex(IndexDesc index);
-
-  boolean existIndex(String indexName);
-
-  boolean existIndex(String tableName, String columnName);
-
-  IndexDesc getIndex(String indexName);
-
-  IndexDesc getIndex(String tableName, String columnName);
-
-  boolean deleteIndex(String indexName);
-
-  boolean registerFunction(FunctionDesc funcDesc);
-
-  boolean unregisterFunction(String signature, DataType... paramTypes);
-
-  /**
-   *
-   * @param signature
-   * @return
-   */
-  FunctionDesc getFunction(String signature, DataType... paramTypes);
-
-  /**
-   *
-   * @param signature
-   * @return
-   */
-  boolean containFunction(String signature, DataType... paramTypes);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
index 4853945..7ea2782 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "tajo.catalog";
+option java_package = "org.apache.tajo.catalog";
 option java_outer_classname = "CatalogProtocol";
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
index acb89fe..60cbbcd 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "tajo.catalog.proto";
+option java_package = "org.apache.tajo.catalog.proto";
 option java_outer_classname = "CatalogProtos";
 option optimize_for = SPEED;
 option java_generic_services = false;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/proto/DataTypes.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/proto/DataTypes.proto b/tajo-catalog/tajo-catalog-client/src/main/proto/DataTypes.proto
index c8b18cb..5fef1ad 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/DataTypes.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/DataTypes.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "tajo.common";
+option java_package = "org.apache.tajo.common";
 option java_outer_classname = "TajoDataTypes";
 option optimize_for = SPEED;
 option java_generic_services = false;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/proto/PrimitiveProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/proto/PrimitiveProtos.proto b/tajo-catalog/tajo-catalog-client/src/main/proto/PrimitiveProtos.proto
index 5fa6903..e722190 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/PrimitiveProtos.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/PrimitiveProtos.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "tajo.rpc.protocolrecords";
+option java_package = "org.apache.tajo.rpc.protocolrecords";
 option java_outer_classname = "PrimitiveProtos";
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-client/src/main/resources/catalog-default.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/resources/catalog-default.xml b/tajo-catalog/tajo-catalog-client/src/main/resources/catalog-default.xml
index 98b7790..9426760 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/resources/catalog-default.xml
+++ b/tajo-catalog/tajo-catalog-client/src/main/resources/catalog-default.xml
@@ -27,7 +27,7 @@
 
 <property>
   <name>tajo.catalog.store.class</name>
-  <value>tajo.catalog.store.DBStore</value>
+  <value>org.apache.tajo.catalog.store.DBStore</value>
 </property>
 
 <property>

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

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/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
new file mode 100644
index 0000000..6625c7d
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.catalog;
+
+public class CatalogConstants {
+  public static final String STORE_CLASS="tajo.catalog.store.class";
+  
+  public static final String JDBC_DRIVER="tajo.catalog.jdbc.driver";
+  public static final String DEFAULT_JDBC_DRIVER="org.apache.derby.jdbc.EmbeddedDriver";
+  
+  public static final String JDBC_URI="tajo.catalog.jdbc.uri";
+
+  private CatalogConstants() {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/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
new file mode 100644
index 0000000..3eb08da
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@ -0,0 +1,181 @@
+/**
+ * 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;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
+import org.apache.tajo.catalog.statistics.TableStat;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.util.FileUtil;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import static org.apache.tajo.common.TajoDataTypes.Type;
+
+public class CatalogUtil {
+  public static String getCanonicalName(String signature,
+      Collection<DataType> paramTypes) {
+    DataType [] types = paramTypes.toArray(new DataType[paramTypes.size()]);
+    return getCanonicalName(signature, types);
+  }
+  public static String getCanonicalName(String signature,
+      DataType...paramTypes) {
+    StringBuilder sb = new StringBuilder(signature);
+    sb.append("(");
+    int i = 0;
+    for (DataType type : paramTypes) {
+      sb.append(type.getType());
+      if(i < paramTypes.length - 1) {
+        sb.append(",");
+      }
+      
+      i++;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public static String prettyPrint(TableMeta meta) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("store type: ").append(meta.getStoreType()).append("\n");
+    sb.append("schema: \n");
+
+    for(int i = 0; i < meta.getSchema().getColumnNum(); i++) {
+      Column col = meta.getSchema().getColumn(i);
+      sb.append(col.getColumnName()).append("\t").append(col.getDataType());
+      sb.append("\n");
+    }
+    return sb.toString();
+  }
+
+  public static void printTableMeta(File file) throws IOException {
+    CatalogProtos.TableProto proto = (CatalogProtos.TableProto) FileUtil.
+        loadProto(file, CatalogProtos.TableProto.getDefaultInstance());
+    System.out.println(prettyPrint(new TableMetaImpl(proto)));
+  }
+
+  public static void main(String [] args) throws IOException {
+    if (args.length < 2) {
+      System.out.println("catalog print [filename]");
+      System.exit(-1);
+    }
+
+    File file = new File(args[1]);
+    printTableMeta(file);
+  }
+
+  public static StoreType getStoreType(final String typeStr) {
+    if (typeStr.equalsIgnoreCase(StoreType.CSV.name())) {
+      return StoreType.CSV;
+    } else if (typeStr.equalsIgnoreCase(StoreType.RAW.name())) {
+      return StoreType.RAW;
+    } else if (typeStr.equalsIgnoreCase(StoreType.CSV.name())) {
+      return StoreType.CSV;
+    } else if (typeStr.equalsIgnoreCase(StoreType.ROWFILE.name())) {
+      return StoreType.ROWFILE;
+    }else if (typeStr.equalsIgnoreCase(StoreType.RCFILE.name())) {
+      return StoreType.RCFILE;
+    } else if (typeStr.equalsIgnoreCase(StoreType.TREVNI.name())) {
+      return StoreType.TREVNI;
+    } else {
+      return null;
+    }
+  }
+
+  public static TableMeta newTableMeta(Schema schema, StoreType type) {
+    return new TableMetaImpl(schema, type, new Options());
+  }
+
+  public static TableMeta newTableMeta(Schema schema, StoreType type,
+      Options options) {
+    return new TableMetaImpl(schema, type, options);
+  }
+
+  public static TableMeta newTableMeta(Schema schema, StoreType type, Options options,
+      TableStat stat) {
+    return new TableMetaImpl(schema, type, options, stat);
+  }
+
+  public static TableDesc newTableDesc(String tableName, TableMeta meta,
+      Path path) {
+    return new TableDescImpl(tableName, meta, path);
+  }
+
+  public static TableDesc newTableDesc(TableDescProto proto) {
+    return new TableDescImpl(proto);
+  }
+
+  public static TableDesc newTableDesc(String tableName,
+      Schema schema, StoreType type, Options options, Path path) {
+    return new TableDescImpl(tableName, schema, type, options, path);
+  }
+
+  /**
+  * This method transforms the unqualified names of a given schema into
+  * the qualified names.
+  *
+  * @param tableName a table name to be prefixed
+  * @param schema a schema to be transformed
+  *
+  * @return
+  */
+  public static SchemaProto getQualfiedSchema(String tableName,
+      SchemaProto schema) {
+    SchemaProto.Builder revisedSchema = SchemaProto.newBuilder(schema);
+    revisedSchema.clearFields();
+    String[] split;
+    for (ColumnProto col : schema.getFieldsList()) {
+      split = col.getColumnName().split("\\.");
+      if (split.length == 1) { // if not qualified name
+        // rewrite the column
+        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
+        builder.setColumnName(tableName + "." + col.getColumnName());
+        col = builder.build();
+      } else if (split.length == 2) {
+        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
+        builder.setColumnName(tableName + "." + split[1]);
+        col = builder.build();
+      } else {
+        throw new InternalError("Unaccetable field name "
+            + col.getColumnName());
+      }
+      revisedSchema.addFields(col);
+    }
+
+    return revisedSchema.build();
+  }
+
+  public static DataType newDataTypeWithoutLen(Type type) {
+    return DataType.newBuilder().setType(type).build();
+  }
+
+  public static DataType [] newDataTypesWithoutLen(Type... types) {
+    DataType [] dataTypes = new DataType[types.length];
+    for (int i = 0; i < types.length; i++) {
+      dataTypes[i] = DataType.newBuilder().setType(types[i]).build();
+    }
+    return dataTypes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
new file mode 100644
index 0000000..e1adc53
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
@@ -0,0 +1,187 @@
+/**
+ * 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;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.json.GsonCreator;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProtoOrBuilder;
+import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class Column implements ProtoObject<ColumnProto>, Cloneable {
+	private ColumnProto proto = ColumnProto.getDefaultInstance();
+	private ColumnProto.Builder builder = null;
+	private boolean viaProto = false;
+	
+	@Expose protected String name;
+	@Expose protected DataType dataType;
+	
+	public Column() {
+		this.builder = ColumnProto.newBuilder();
+	}
+	  
+	public Column(String columnName, DataType dataType) {
+	  this();
+		this.name = columnName.toLowerCase();
+		this.dataType = dataType;
+	}
+
+  public Column(String columnName, TajoDataTypes.Type type) {
+    this(columnName, CatalogUtil.newDataTypeWithoutLen(type));
+  }
+	
+	public Column(ColumnProto proto) {
+		this.proto = proto;
+		this.viaProto = true;
+	}
+	
+
+	public String getQualifiedName() {
+		ColumnProtoOrBuilder p = viaProto ? proto : builder;
+		if(name != null) {
+			return this.name;
+		}
+		if(!p.hasColumnName()) {
+			return null;			
+		}		
+		this.name = p.getColumnName();
+		
+		return this.name;
+	}
+	
+  public boolean isQualified() {
+    return getQualifiedName().split("\\.").length == 2;
+  }
+
+  public String getTableName() {
+    if (isQualified()) {
+      return getQualifiedName().split("\\.")[0];
+    } else {
+      return "";
+    }    
+  }
+
+  public String getColumnName() {
+    if (isQualified())
+      return getQualifiedName().split("\\.")[1];
+    else
+      return getQualifiedName();
+  }
+	
+	public void setName(String name) {
+	  setModified();
+		this.name = name.toLowerCase();
+	}
+	
+	public DataType getDataType() {
+		ColumnProtoOrBuilder p = viaProto ? proto : builder;
+		if(dataType != null) {
+			return this.dataType;
+		}
+		if(!p.hasDataType()) {
+			return null;
+		}
+		this.dataType = p.getDataType();
+		
+		return this.dataType;
+	}
+	
+	public void setDataType(DataType dataType) {
+		setModified();
+		this.dataType = dataType;
+	}
+	
+	@Override
+	public boolean equals(Object o) {
+		if (o instanceof Column) {
+			Column cd = (Column)o;
+			return this.getQualifiedName().equals(cd.getQualifiedName()) &&
+					this.getDataType().equals(cd.getDataType());
+		}
+		return false;
+	}
+	
+  public int hashCode() {
+    return getQualifiedName().hashCode() ^ (getDataType().hashCode() * 17);
+  }
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    Column column = (Column) super.clone();
+    initFromProto();
+    column.proto = null;
+    column.viaProto = false;
+    column.builder = ColumnProto.newBuilder();
+    column.name = name;
+    column.dataType = dataType;
+    return column;
+  }
+
+	@Override
+	public ColumnProto getProto() {
+	  if(!viaProto) {
+      mergeLocalToBuilder();
+      proto = builder.build();
+      viaProto = true;
+    }
+	  
+	  return proto;
+	}
+	
+	private void setModified() {
+	  if (viaProto && builder == null) {
+	    builder = ColumnProto.newBuilder(proto);
+	  }
+	  viaProto = false;
+	}
+	
+	private void mergeLocalToBuilder() {
+	  if (builder == null) {
+	    builder = ColumnProto.newBuilder(proto);
+	  }
+		if (this.name != null) {
+			builder.setColumnName(this.name);			
+		}
+		if (this.dataType != null) {
+			builder.setDataType(this.dataType);
+		}
+	}
+	
+	public String toString() {
+	  return getQualifiedName() +" (" + getDataType().getType() +")";
+	}
+	
+	public String toJSON() {
+		initFromProto();
+		return GsonCreator.getInstance().toJson(this);
+	}
+
+	@Override
+	public void initFromProto() {
+		ColumnProtoOrBuilder p = viaProto ? proto : builder;
+		if (this.name == null && p.hasColumnName()) {
+			this.name = p.getColumnName();
+		}
+		if (this.dataType == null && p.hasDataType()) {
+			this.dataType = p.getDataType();
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
new file mode 100644
index 0000000..93bcadd
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
@@ -0,0 +1,274 @@
+/**
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.function.Function;
+import org.apache.tajo.catalog.function.GeneralFunction;
+import org.apache.tajo.catalog.json.GsonCreator;
+import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProtoOrBuilder;
+import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
+import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.exception.InternalException;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+
+public class FunctionDesc implements ProtoObject<FunctionDescProto>, Cloneable {
+  private FunctionDescProto proto = FunctionDescProto.getDefaultInstance();
+  private FunctionDescProto.Builder builder = null;
+  private boolean viaProto = false;
+  
+  @Expose private String signature;
+  @Expose private Class<? extends Function> funcClass;
+  @Expose private FunctionType funcType;
+  @Expose private DataType [] returnType;
+  @Expose private DataType [] params;
+
+  public FunctionDesc() {
+    this.builder = FunctionDescProto.newBuilder();
+  }
+  
+  public FunctionDesc(String signature, Class<? extends Function> clazz,
+      FunctionType funcType, DataType [] retType, DataType [] params) {
+    this();
+    this.signature = signature.toLowerCase();
+    this.funcClass = clazz;
+    this.funcType = funcType;
+    this.returnType = retType;
+    this.params = params;
+  }
+  
+  public FunctionDesc(FunctionDescProto proto) {
+    this.proto = proto;
+    this.viaProto = true;
+  }
+
+  @SuppressWarnings("unchecked")
+  public FunctionDesc(String signature, String className, FunctionType type,
+                      DataType [] retType, DataType... argTypes) throws ClassNotFoundException {
+    this(signature, (Class<? extends Function>) Class.forName(className), type,
+        retType, argTypes);
+  }
+
+  /**
+   * 
+   * @return 함수 인스턴스
+   * @throws InternalException
+   */
+  public Function newInstance() throws InternalException {
+    try {
+      Constructor<? extends Function> cons = getFuncClass().getConstructor();
+      return cons.newInstance();
+    } catch (Exception ioe) {
+      throw new InternalException("Cannot initiate function " + signature);
+    }
+  }
+
+  public String getSignature() {
+    FunctionDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.signature != null) {
+      return this.signature;
+    }
+    if (!proto.hasSignature()) {
+      return null;
+    }
+    this.signature = p.getSignature();
+    return this.signature;
+  }
+
+  @SuppressWarnings("unchecked")
+  public Class<? extends Function> getFuncClass() throws InternalException {
+    FunctionDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.funcClass != null) {
+      return this.funcClass;
+    }
+    if (!p.hasClassName()) {
+      return null;
+    }
+    try {
+      this.funcClass = (Class<? extends Function>)Class.forName(p.getClassName());
+    } catch (ClassNotFoundException e) {
+      throw new InternalException("The function class ("+p.getClassName()+") cannot be loaded");
+    }
+    return this.funcClass;
+  }
+
+  public FunctionType getFuncType() {
+    FunctionDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.funcType != null) {
+      return this.funcType;
+    }
+    if (!p.hasType()) {
+      return null;
+    }
+    this.funcType = p.getType();
+    return this.funcType;
+  }
+
+  public DataType [] getParamTypes() {
+    FunctionDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.params != null) {
+      return this.params;
+    }
+    if (p.getParameterTypesCount() == 0) {
+      return null;
+    }
+    this.params = p.getParameterTypesList().toArray(
+        new DataType[p.getParameterTypesCount()]);
+    return this.params;
+  }
+
+  public DataType [] getReturnType() {
+    FunctionDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.returnType != null) {
+      return newNoNameSchema(this.returnType);
+    }
+    if (!p.hasReturnType()) {
+      return null;
+    }
+    this.returnType = newNoNameSchema(p.getReturnType());
+    return this.returnType;
+    
+  }
+
+  public static DataType [] newNoNameSchema(DataType ... types) {
+    DataType [] dataTypes = types.clone();
+    return dataTypes;
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof FunctionDesc) {
+      FunctionDesc other = (FunctionDesc) obj;
+      if(this.getProto().equals(other.getProto()))
+        return true;
+    }
+    return false;
+  }
+  
+  @Override
+  public Object clone() throws CloneNotSupportedException{
+    FunctionDesc desc  = (FunctionDesc)super.clone();
+    desc.proto = this.proto;
+    desc.builder = this.builder == null?null:this.builder.clone();
+    
+    desc.signature = this.signature;
+    desc.params = this.params;
+    
+    desc.returnType = this.returnType;
+    desc.viaProto = this.viaProto;
+    desc.funcClass = this.funcClass;
+    
+    return desc;
+  }
+
+  @Override
+  public FunctionDescProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+  
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = FunctionDescProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+  
+  private void mergeLocalToBuilder() {
+    if (this.signature  != null) {     
+      builder.setSignature(this.signature);
+    }
+    if (this.funcClass != null) {
+      builder.setClassName(this.funcClass.getName());
+    }
+    if (this.funcType != null) {
+      builder.setType(this.funcType);
+    }
+    if (this.returnType != null) {
+      builder.setReturnType(this.returnType[0]);
+    }
+    if (this.params != null) {
+      builder.addAllParameterTypes(Arrays.asList(params));
+    }
+  }
+  
+  private void mergeLocalToProto() {
+    if(viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+  
+  @SuppressWarnings("unchecked")
+  private void mergeProtoToLocal() throws InternalException {
+	  FunctionDescProtoOrBuilder p = viaProto ? proto : builder;
+	  if (signature == null && p.hasSignature()) {
+		  signature = p.getSignature();
+	  }
+	  if (funcClass == null && p.hasClassName()) {
+		  try {
+			  this.funcClass = 
+			      (Class<? extends GeneralFunction>)Class.forName(p.getClassName());
+		  } catch (ClassNotFoundException e) {
+			  throw new InternalException("The function class ("+p.getClassName()+") cannot be loaded");
+		  }
+	  }
+	  if (funcType == null && p.hasType()) {
+		  funcType = p.getType();
+	  }
+	  if (returnType == null && p.hasReturnType()) {
+		  returnType = newNoNameSchema(p.getReturnType());
+	  }
+	  if (params == null && p.getParameterTypesCount() > 0) {
+		  params = new DataType[p.getParameterTypesCount()];
+		  for (int i = 0; i < p.getParameterTypesCount(); i++) {
+			  params[i] = p.getParameterTypes(i);
+		  }
+	  }
+  }
+  
+  @Override
+  public String toString() {
+	  return getProto().toString();
+  }
+
+  @Override
+  public void initFromProto() {
+    try {
+      mergeProtoToLocal();
+    } catch (InternalException e) {
+      e.printStackTrace();
+    }
+  }
+  
+  public String toJSON() {
+    initFromProto();
+    Gson gson = GsonCreator.getInstance();
+    return gson.toJson(this, FunctionDesc.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java
new file mode 100644
index 0000000..fb6058e
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java
@@ -0,0 +1,265 @@
+/**
+ * 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;
+
+import com.google.common.base.Objects;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProtoOrBuilder;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
+import org.apache.tajo.common.ProtoObject;
+
+public class IndexDesc implements ProtoObject<IndexDescProto>, Cloneable {
+  private IndexDescProto proto;
+  private IndexDescProto.Builder builder;
+  private boolean viaProto;
+  
+  @Expose private String name;
+  @Expose private String tableId;
+  @Expose private Column column;
+  @Expose private IndexMethod indexMethod;
+  @Expose private boolean isUnique = false;
+  @Expose private boolean isClustered = false;
+  @Expose private boolean isAscending = false;
+  
+  public IndexDesc() {
+    this.builder = IndexDescProto.newBuilder();
+  }
+  
+  public IndexDesc(String name, String tableId, Column column, IndexMethod type,
+      boolean isUnique, boolean isClustered, boolean isAscending) {
+    this();
+    this.name = name.toLowerCase();
+    this.tableId = tableId.toLowerCase();
+    this.column = column;
+    this.indexMethod = type;
+    this.isUnique = isUnique;
+    this.isClustered = isClustered;
+    this.isAscending = isAscending;
+  }
+  
+  public IndexDesc(IndexDescProto proto) {
+    this.proto = proto;
+    this.viaProto = true;
+  }
+  
+  public String getName() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.name != null) {
+      return name;
+    }
+    if (!p.hasName()) {
+      return null;
+    }
+    this.name = p.getName();
+    
+    return name;
+  }
+  
+  public String getTableId() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.tableId != null) {
+      return tableId;
+    }
+    if (!p.hasTableId()) {
+      return null;
+    }
+    this.tableId = p.getTableId();
+    
+    return tableId;
+  }
+  
+  public Column getColumn() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.column != null) {
+      return column;
+    }
+    if (!p.hasColumn()) {
+      return null;
+    }
+    this.column = new Column(p.getColumn());
+    
+    return column;
+  }
+  
+  public IndexMethod getIndexMethod() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;    
+    if (this.indexMethod != null) {
+      return this.indexMethod;
+    }
+    if (!p.hasIndexMethod()) { // if isCluster == false and proto has no set
+      return null;
+    }
+    this.indexMethod = p.getIndexMethod();
+    
+    return this.indexMethod;
+  }
+  
+  public boolean isClustered() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (isClustered) {
+      return true;
+    }
+    if (!p.hasIsClustered()) { // if isCluster == false and proto has no set
+      return false;
+    }
+    this.isClustered = p.getIsClustered();
+    
+    return this.isClustered;
+  }
+  
+  public boolean isUnique() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;    
+    if (isUnique) {
+      return true;
+    }
+    if (!p.hasIsUnique()) { // if isCluster == false and proto has no set
+      return false;
+    }
+    this.isUnique = p.getIsUnique();
+    
+    return this.isUnique;
+  }
+  
+  public boolean isAscending() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;    
+    if (isAscending) {
+      return true;
+    }
+    if (!p.hasIsAscending()) { // if isCluster == false and proto has no set
+      return false;
+    }
+    this.isAscending = p.getIsAscending();
+    
+    return this.isAscending;
+  }
+
+  @Override
+  public IndexDescProto getProto() {
+    if(!viaProto) {
+      mergeLocalToBuilder();
+      proto = builder.build();
+      viaProto = true;
+    }
+    
+    return proto;
+  }
+  
+  @SuppressWarnings("unused")
+  private void setModified() {
+    if (viaProto && builder == null) {
+      builder = IndexDescProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+  
+  private void mergeLocalToBuilder() {
+    if (builder == null) {
+      builder = IndexDescProto.newBuilder(proto);
+    }
+    if (this.name != null) {
+      builder.setName(this.name);
+    }
+    if (this.tableId != null) {
+      builder.setTableId(this.tableId);
+    }
+    if (this.column != null) {
+      builder.setColumn(this.column.getProto());
+    }
+    if (this.indexMethod != null) {
+      builder.setIndexMethod(indexMethod);
+    }
+    if (this.isUnique) {
+      builder.setIsUnique(this.isUnique);
+    }
+    if (this.isClustered) {
+      builder.setIsClustered(this.isClustered);
+    }
+    if (this.isAscending) {
+      builder.setIsAscending(this.isAscending);
+    }
+  }
+  
+  @Override
+  public void initFromProto() {
+    IndexDescProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.name == null && p.hasName()) {
+      this.name = p.getName();
+    }
+    if (this.tableId == null && p.hasTableId()) {
+      this.tableId = p.getTableId();
+    }
+    if (this.column == null && p.hasColumn()) {
+      this.column = new Column(p.getColumn());
+    }
+    if (this.indexMethod == null && p.hasIndexMethod()) {
+      this.indexMethod = p.getIndexMethod();
+    }
+    if (this.isUnique == false && p.hasIsUnique()) {
+      this.isUnique = p.getIsUnique();
+    }
+    if (this.isClustered == false && p.hasIsClustered()) {
+      this.isUnique = p.getIsUnique();
+    }
+    if (this.isAscending == false && p.hasIsAscending()) {
+      this.isAscending = p.getIsAscending();
+    }
+    viaProto = false;
+  }
+  
+  public boolean equals(Object obj) {
+    if (obj instanceof IndexDesc) {
+      IndexDesc other = (IndexDesc) obj;
+      return getName().equals(other.getName())
+          && getTableId().equals(other.getTableId())
+          && getColumn().equals(other.getColumn())
+          && getIndexMethod().equals(other.getIndexMethod())
+          && isUnique() == other.isUnique()
+          && isClustered() == other.isClustered()
+          && isAscending() == other.isAscending();
+    } else {
+      return false;
+    }
+  }
+  
+  public int hashCode() {
+    return Objects.hashCode(getName(), getTableId(), getColumn(), 
+        getIndexMethod(), isUnique(), isClustered(), isAscending());
+  }
+  
+  public Object clone() throws CloneNotSupportedException {
+    IndexDesc desc = (IndexDesc) super.clone();
+    initFromProto();
+    desc.name = name;
+    desc.tableId = tableId;
+    desc.column = (Column) column.clone();
+    desc.indexMethod = indexMethod;
+    desc.isUnique = isUnique;
+    desc.isClustered = isClustered;
+    desc.isAscending = isAscending;
+    return desc; 
+  }
+  
+  public String toString() {
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    return gson.toJson(this);
+  }
+}
\ No newline at end of file