You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by bl...@apache.org on 2014/03/04 15:42:50 UTC

git commit: TAJO-641: NPE in HCatalogStore.addTable(). (jaehwa)

Repository: incubator-tajo
Updated Branches:
  refs/heads/master e47242e68 -> e0b6f7ae8


TAJO-641: NPE in HCatalogStore.addTable(). (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/e0b6f7ae
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/e0b6f7ae
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/e0b6f7ae

Branch: refs/heads/master
Commit: e0b6f7ae8e2215e71500d1d0d8faa2318d3d3d1e
Parents: e47242e
Author: blrunner <jh...@gruter.com>
Authored: Tue Mar 4 23:42:38 2014 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Tue Mar 4 23:42:38 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../java/org/apache/tajo/catalog/Schema.java    |   5 +-
 .../tajo/catalog/store/HCatalogStore.java       |  80 +++-
 .../catalog/store/HCatalogStoreClientPool.java  |  16 +-
 .../tajo/catalog/store/DummyListener.java       |  97 -----
 .../tajo/catalog/store/TestHCatalogStore.java   | 363 +++++++++++--------
 6 files changed, 297 insertions(+), 266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e0b6f7ae/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 09d2fb5..ecd5eb9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -269,6 +269,8 @@ Release 0.8.0 - unreleased
 
   BUG FIXES
 
+    TAJO-641: NPE in HCatalogStore.addTable(). (jaehwa)
+
     TAJO-646: TajoClient is blocked while main thread finished.
     (hyoungjunkim via jinho)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e0b6f7ae/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 725e9c3..0c812f1 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
@@ -130,7 +130,10 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
    */
   public Column getColumn(String name) {
     String [] parts = name.split("\\.");
-    if (parts.length == 2) {
+    // Some of the string can includes database name and table name and column name.
+    // For example, it can be 'default.table1.id'.
+    // Therefore, spilt string array length can be 3.
+    if (parts.length >= 2) {
       return getColumnByQName(name);
     } else {
       return getColumnByName(name);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e0b6f7ae/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
index 3dc0908..d618375 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.catalog.store;
 
+import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -39,6 +40,9 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.exception.InternalException;
 
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -47,17 +51,31 @@ import java.util.Properties;
 import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType;
 
 public class HCatalogStore extends CatalogConstants implements CatalogStore {
-  public static final String CVSFILE_DELIMITER = "csvfile.delimiter";
+  public static final String CSVFILE_DELIMITER = "csvfile.delimiter";
+  public static final String CSVFILE_NULL = "csvfile.null";
 
   protected final Log LOG = LogFactory.getLog(getClass());
   protected Configuration conf;
   private static final int CLIENT_POOL_SIZE = 2;
   private final HCatalogStoreClientPool clientPool = new HCatalogStoreClientPool(0);
+  private ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
 
   public HCatalogStore(final Configuration conf)
       throws InternalException {
     this.conf = conf;
     try {
+      // In unit test case, HCatalogStore has to set some hive configurations by force.
+      // So, it checks caller class name and caller method name.
+      long[] threadIds = threadBean.getAllThreadIds();
+      for (long tid : threadIds) {
+        ThreadInfo info = threadBean.getThreadInfo(tid, 5);
+        for (StackTraceElement frame : info.getStackTrace()) {
+          if (frame.toString().equals("org.apache.tajo.catalog.store.TestHCatalogStore.setUp")) {
+            clientPool.setParameters(this.conf);
+          }
+        }
+      }
+
       clientPool.addClients(CLIENT_POOL_SIZE);
     } catch (Exception e) {
       e.printStackTrace();
@@ -179,21 +197,28 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
       Properties properties = table.getMetadata();
       if (properties != null) {
         // set field delimiter
-        String fieldDelimiter = "", fileOutputformat = "";
+        String fieldDelimiter = "", fileOutputformat = "", nullFormat = "";
         if (properties.getProperty(serdeConstants.FIELD_DELIM) != null) {
           fieldDelimiter = properties.getProperty(serdeConstants.FIELD_DELIM);
         } else {
           // if hive table used default row format delimiter, Properties doesn't have it.
           // So, Tajo must set as follows:
-          fieldDelimiter = "\\001";
+          fieldDelimiter = "\u0001";
+        }
+
+        // set null format
+        if (properties.getProperty(serdeConstants.SERIALIZATION_NULL_FORMAT) != null) {
+          nullFormat = properties.getProperty(serdeConstants.SERIALIZATION_NULL_FORMAT);
+        } else {
+          nullFormat = "\\N";
         }
 
         // set file output format
         fileOutputformat = properties.getProperty("file.outputformat");
         storeType = CatalogUtil.getStoreType(HCatalogUtil.getStoreType(fileOutputformat));
-
         if (storeType.equals(CatalogProtos.StoreType.CSV) ) {
-          options.put(CVSFILE_DELIMITER, fieldDelimiter);
+          options.put(CSVFILE_DELIMITER, StringEscapeUtils.escapeJava(fieldDelimiter));
+          options.put(CSVFILE_NULL, StringEscapeUtils.escapeJava(nullFormat));
         }
 
         // set data size
@@ -310,46 +335,61 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
       //table.setOwner();
 
       StorageDescriptor sd = new StorageDescriptor();
+      sd.setParameters(new HashMap<String, String>());
+      sd.setSerdeInfo(new SerDeInfo());
+      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
+      sd.getSerdeInfo().setName(table.getTableName());
 
       // if tajo set location method, thrift client make exception as follows:
       // Caused by: MetaException(message:java.lang.NullPointerException)
       // If you want to modify table path, you have to modify on Hive cli.
-      //sd.setLocation(tableDesc.getPath().toString());
+      // sd.setLocation(tableDesc.getPath().toString());
 
       // set column information
       ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(tableDesc.getSchema().getFieldsCount());
-      for (CatalogProtos.ColumnProto col : tableDesc.getSchema().getFieldsList()) {
-        cols.add(new FieldSchema(
-            col.getName(),
-            HCatalogUtil.getHiveFieldType(col.getDataType().getType().name()),
-            ""));
+
+      for (CatalogProtos.ColumnProto eachField : tableDesc.getSchema().getFieldsList()) {
+        cols.add(new FieldSchema( eachField.getName(), HCatalogUtil.getHiveFieldType(eachField.getDataType().getType().name()), ""));
       }
       sd.setCols(cols);
 
+      // set partition keys
+      if (tableDesc.getPartition() != null && tableDesc.getPartition().getPartitionType().equals(PartitionType.COLUMN)) {
+        List<FieldSchema> partitionKeys = new ArrayList<FieldSchema>();
+        for(CatalogProtos.ColumnProto eachPartitionKey: tableDesc.getPartition().getExpressionSchema().getFieldsList()) {
+          partitionKeys.add(new FieldSchema( eachPartitionKey.getName(), HCatalogUtil.getHiveFieldType(eachPartitionKey.getDataType().getType().name()), ""));
+        }
+        table.setPartitionKeys(partitionKeys);
+      }
+
       sd.setCompressed(false);
+
       if (tableDesc.getMeta().hasParams()) {
         for (CatalogProtos.KeyValueProto entry: tableDesc.getMeta().getParams().getKeyvalList()) {
           if (entry.getKey().equals("compression.codec")) {
             sd.setCompressed(true);
-          } else if (entry.getKey().equals(CVSFILE_DELIMITER)) {
-            sd.getSerdeInfo().getParameters().put(serdeConstants.FIELD_DELIM, entry.getValue());
+          } else if (entry.getKey().equals(CSVFILE_NULL)) {
+            sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_NULL_FORMAT, StringEscapeUtils.unescapeJava(entry.getValue()));
+          } else if (entry.getKey().equals(CSVFILE_DELIMITER)) {
+            String fieldDelimiter = entry.getValue();
+
+            // User can use an unicode for filed delimiter such as \u0001, \001.
+            // In this case, java console will convert this value into "\\u001".
+            // And hive will un-espace this value again.
+            // As a result, user can use right field delimiter.
+            // So, we have to un-escape this value.
+            sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, StringEscapeUtils.unescapeJava(fieldDelimiter));
+            sd.getSerdeInfo().getParameters().put(serdeConstants.FIELD_DELIM, StringEscapeUtils.unescapeJava(fieldDelimiter));
           }
         }
       }
 
-      sd.setParameters(new HashMap<String, String>());
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(table.getTableName());
-
       if(tableDesc.getMeta().getStoreType().equals(CatalogProtos.StoreType.RCFILE)) {
         sd.getSerdeInfo().setSerializationLib(org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe.class.getName());
       } else {
         sd.getSerdeInfo().setSerializationLib(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
       }
 
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-//      sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1");
-
       if(tableDesc.getMeta().getStoreType().equals(CatalogProtos.StoreType.RCFILE)) {
         sd.setInputFormat(org.apache.hadoop.hive.ql.io.RCFileInputFormat.class.getName());
       } else {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e0b6f7ae/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStoreClientPool.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStoreClientPool.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStoreClientPool.java
index 0a6c9b9..0f0ea04 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStoreClientPool.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStoreClientPool.java
@@ -15,10 +15,14 @@ package org.apache.tajo.catalog.store;
 
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.log4j.Logger;
 
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -31,7 +35,7 @@ public class HCatalogStoreClientPool {
   private final ConcurrentLinkedQueue<HCatalogStoreClient> clientPool =
       new ConcurrentLinkedQueue<HCatalogStoreClient>();
   private AtomicBoolean poolClosed = new AtomicBoolean(false);
-  private final HiveConf hiveConf;
+  private HiveConf hiveConf;
 
   /**
    * A wrapper around the HiveMetaStoreClient that manages interactions with the
@@ -43,9 +47,7 @@ public class HCatalogStoreClientPool {
 
     private HCatalogStoreClient(HiveConf hiveConf) {
       try {
-
         LOG.info("Creating MetaStoreClient. Pool Size = " + clientPool.size());
-
         this.hiveClient = new HiveMetaStoreClient(hiveConf);
       } catch (Exception e) {
         // Turn in to an unchecked exception
@@ -99,6 +101,13 @@ public class HCatalogStoreClientPool {
     addClients(initialSize);
   }
 
+  public void setParameters(Configuration conf) {
+    for( Iterator<Entry<String, String>> iter = conf.iterator(); iter.hasNext();) {
+      Map.Entry<String, String> entry = iter.next();
+      this.hiveConf.set(entry.getKey(), entry.getValue());
+    }
+  }
+
   /**
    * Add numClients to the client pool.
    */
@@ -127,6 +136,7 @@ public class HCatalogStoreClientPool {
       client = new HCatalogStoreClient(hiveConf);
     }
     client.markInUse();
+
     return client;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e0b6f7ae/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/DummyListener.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/DummyListener.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/DummyListener.java
deleted file mode 100644
index a3c2dca..0000000
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/DummyListener.java
+++ /dev/null
@@ -1,97 +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.catalog.store;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.MetaStoreEventListener;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.events.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/** A dummy implementation for
- * {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener}
- * for testing purposes.
- */
-public class DummyListener extends MetaStoreEventListener {
-
-  public static final List<ListenerEvent> notifyList = new ArrayList<ListenerEvent>();
-
-  /**
-   * @return The last event received, or null if no event was received.
-   */
-  public static ListenerEvent getLastEvent() {
-    if (notifyList.isEmpty()) {
-      return null;
-    } else {
-      return notifyList.get(notifyList.size() - 1);
-    }
-  }
-
-  public DummyListener(Configuration config) {
-    super(config);
-  }
-
-  @Override
-  public void onAddPartition(AddPartitionEvent partition) throws MetaException {
-    notifyList.add(partition);
-  }
-
-  @Override
-  public void onCreateDatabase(CreateDatabaseEvent db) throws MetaException {
-    notifyList.add(db);
-  }
-
-  @Override
-  public void onCreateTable(CreateTableEvent table) throws MetaException {
-    notifyList.add(table);
-  }
-
-  @Override
-  public void onDropDatabase(DropDatabaseEvent db) throws MetaException {
-    notifyList.add(db);
-  }
-
-  @Override
-  public void onDropPartition(DropPartitionEvent partition) throws MetaException {
-    notifyList.add(partition);
-  }
-
-  @Override
-  public void onDropTable(DropTableEvent table) throws MetaException {
-    notifyList.add(table);
-  }
-
-  @Override
-  public void onAlterTable(AlterTableEvent event) throws MetaException {
-    notifyList.add(event);
-  }
-
-  @Override
-  public void onAlterPartition(AlterPartitionEvent event) throws MetaException {
-    notifyList.add(event);
-  }
-
-  @Override
-  public void onLoadPartitionDone(LoadPartitionDoneEvent partEvent) throws MetaException {
-    notifyList.add(partEvent);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e0b6f7ae/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
index 82a334b..3cf2302 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
@@ -19,20 +19,17 @@
 package org.apache.tajo.catalog.store;
 
 
+import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.api.*;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.tajo.catalog.CatalogConstants;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.TableDesc;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
@@ -42,7 +39,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 import static org.junit.Assert.*;
 
@@ -56,9 +55,14 @@ public class TestHCatalogStore {
   private static final String DB_NAME = "test_hive";
   private static final String CUSTOMER = "customer";
   private static final String NATION = "nation";
+  private static final String REGION = "region";
+  private static final String SUPPLIER = "supplier";
+
   private static CatalogStore store;
   private static int port;
 
+  private static Path warehousePath;
+
   @BeforeClass
   public static void setUp() throws Exception {
     // delete metstore default path for successful unit tests
@@ -68,46 +72,30 @@ public class TestHCatalogStore {
     Database db = new Database();
     db.setName(DB_NAME);
 
-    Path warehousePath = new Path(CommonTestingUtil.getTestDir(), DB_NAME);
-    port = MetaStoreUtils.findFreePort();
-    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
+    warehousePath = new Path(CommonTestingUtil.getTestDir(), DB_NAME);
     db.setLocationUri(warehousePath.toString());
 
-    String metastoreUri = "thrift://localhost:" + port;
-
     HiveConf conf = new HiveConf();
-    conf.setVar(HiveConf.ConfVars.METASTOREURIS, metastoreUri);
-    conf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-    conf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    conf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    conf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    conf.set(HiveConf.ConfVars.METASTORE_EVENT_LISTENERS.varname, DummyListener.class.getName());
-
-    // set property to use at HCatalogUtil
-    System.setProperty(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUri);
-    System.setProperty(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES.varname, "3");
-    System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    System.setProperty(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    System.setProperty(HiveConf.ConfVars.METASTORE_EVENT_LISTENERS.varname, DummyListener.class.getName());
-
-    SessionState.start(new CliSessionState(conf));
+    conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehousePath.toString());
+
+    // create hive configuration file for unit tests
+    Path path = new Path(warehousePath.getParent(), "hive-site.xml");
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    conf.writeXml(fs.create(path));
 
     // create database and tables on Hive MetaStore.
     client = new HiveMetaStoreClient(conf);
-
     client.createDatabase(db);
-    createTable(NATION);
-    createTable(CUSTOMER);
-
-    DummyListener.notifyList.clear();
 
     // create local HCatalogStore.
     TajoConf tajoConf = new TajoConf();
     tajoConf.set(CatalogConstants.STORE_CLASS, HCatalogStore.class.getCanonicalName());
     tajoConf.setVar(TajoConf.ConfVars.CATALOG_ADDRESS, "127.0.0.1:0");
+    tajoConf.addResource(path.toString());
+    tajoConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehousePath.toString());
 
     store = new HCatalogStore(tajoConf);
+
   }
 
   private static void deleteMetaStoreDirectory() throws Exception {
@@ -119,59 +107,6 @@ public class TestHCatalogStore {
     fs.close();
   }
 
-  private static void createTable(String tableName) throws Exception {
-    Map<String, String> tableParams = new HashMap<String, String>();
-
-    List<FieldSchema> cols = new ArrayList<FieldSchema>();
-
-    if (tableName.equals(CUSTOMER)) {
-      cols.add(new FieldSchema("c_custkey", "int", ""));
-      cols.add(new FieldSchema("c_name", "string", ""));
-      cols.add(new FieldSchema("c_address", "string", ""));
-      cols.add(new FieldSchema("c_nationkey", "int", ""));
-      cols.add(new FieldSchema("c_phone", "string", ""));
-      cols.add(new FieldSchema("c_acctbal", "double", ""));
-      cols.add(new FieldSchema("c_mktsegment", "string", ""));
-      cols.add(new FieldSchema("c_comment", "string", ""));
-    } else {
-      cols.add(new FieldSchema("n_nationkey", "int", ""));
-      cols.add(new FieldSchema("n_name", "string", ""));
-      cols.add(new FieldSchema("n_regionkey", "int", ""));
-      cols.add(new FieldSchema("n_comment", "string", ""));
-    }
-
-    StorageDescriptor sd = new StorageDescriptor();
-    sd.setCols(cols);
-    sd.setCompressed(false);
-    sd.setParameters(tableParams);
-    sd.setSerdeInfo(new SerDeInfo());
-    sd.getSerdeInfo().setName(tableName);
-    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-    sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1");
-
-    if (tableName.equals(CUSTOMER)) {
-      sd.getSerdeInfo().setSerializationLib(org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe.class.getName());
-      sd.setInputFormat(org.apache.hadoop.hive.ql.io.RCFileInputFormat.class.getName());
-      sd.setOutputFormat(org.apache.hadoop.hive.ql.io.RCFileOutputFormat.class.getName());
-    } else {
-      sd.setInputFormat(org.apache.hadoop.mapred.TextInputFormat.class.getName());
-      sd.setOutputFormat(org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat.class.getName());
-    }
-
-
-    Table table = new Table();
-    table.setDbName(DB_NAME);
-    table.setTableName(tableName);
-    table.setParameters(tableParams);
-    table.setSd(sd);
-
-    if (tableName.equals(NATION)) {
-      table.addToPartitionKeys(new FieldSchema("type", "string", ""));
-    }
-
-    client.createTable(table);
-  }
-
   @AfterClass
   public static void tearDown() throws IOException {
     try {
@@ -179,6 +114,10 @@ public class TestHCatalogStore {
         store.deleteTable(DB_NAME + "." + CUSTOMER);
       if (store.existTable(DB_NAME + "." + NATION))
         store.deleteTable(DB_NAME + "." + NATION);
+      if (store.existTable(DB_NAME + "." + REGION))
+        store.deleteTable(DB_NAME + "." + REGION);
+      if (store.existTable(DB_NAME + "." + SUPPLIER))
+        store.deleteTable(DB_NAME + "." + SUPPLIER);
       dropDatabase();
       client.close();
       store.close();
@@ -198,56 +137,122 @@ public class TestHCatalogStore {
     }
   }
 
-  // Current Hive MetaStoreClient doesn't support to set table location.
-  // If Tajo update table path, ThriftHiveMetaStore make a MetaException.
-  // So, it needs to disable until Hive support to set table location.
-//  @Test
-//  public void testAddTable() throws Exception {
-//    TableDesc table = new TableDesc();
-//
-//    table.setName(CUSTOMER);
-//
-//    org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
-//    schema.addColumn("c_custkey", TajoDataTypes.Type.INT4);
-//    schema.addColumn("c_name", TajoDataTypes.Type.TEXT);
-//    schema.addColumn("c_address", TajoDataTypes.Type.TEXT);
-//    schema.addColumn("c_nationkey", TajoDataTypes.Type.INT4);
-//    schema.addColumn("c_phone", TajoDataTypes.Type.TEXT);
-//    schema.addColumn("c_acctbal", TajoDataTypes.Type.FLOAT8);
-//    schema.addColumn("c_mktsegment", TajoDataTypes.Type.TEXT);
-//    schema.addColumn("c_comment", TajoDataTypes.Type.TEXT);
-//
-//    table.setSchema(schema);
-//    store.addTable(table);
-//  }
-//  @Test
-//  public void testAddTableByPartition() throws Exception {
-//    TableDesc table = new TableDesc();
-//
-//    table.setName(NATION);
-//
-//    org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
-//    schema.addColumn("n_nationkey", TajoDataTypes.Type.INT4);
-//    schema.addColumn("n_name", TajoDataTypes.Type.TEXT);
-//    schema.addColumn("n_regionkey", TajoDataTypes.Type.INT4);
-//    schema.addColumn("n_comment", TajoDataTypes.Type.TEXT);
-//    table.setSchema(schema);
-//
-//    Partitions partitions = new Partitions();
-//    partitions.addColumn("type", TajoDataTypes.Type.TEXT);
-//    table.setPartitions(partitions);
-//
-//    store.addTable(table);
-//  }
+  @Test
+  public void testAddTable1() throws Exception {
+    TableDesc table = new TableDesc();
+
+    table.setName(DB_NAME + "." + CUSTOMER);
+
+    Options options = new Options();
+    options.put(HCatalogStore.CSVFILE_DELIMITER, "\u0001");
+    TableMeta meta = new TableMeta(CatalogProtos.StoreType.RCFILE, options);
+    table.setMeta(meta);
+
+    table.setPath(new Path(warehousePath, CUSTOMER));
+
+    org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
+    schema.addColumn("c_custkey", TajoDataTypes.Type.INT4);
+    schema.addColumn("c_name", TajoDataTypes.Type.TEXT);
+    schema.addColumn("c_address", TajoDataTypes.Type.TEXT);
+    schema.addColumn("c_nationkey", TajoDataTypes.Type.INT4);
+    schema.addColumn("c_phone", TajoDataTypes.Type.TEXT);
+    schema.addColumn("c_acctbal", TajoDataTypes.Type.FLOAT8);
+    schema.addColumn("c_mktsegment", TajoDataTypes.Type.TEXT);
+    schema.addColumn("c_comment", TajoDataTypes.Type.TEXT);
+
+    table.setSchema(schema);
+    store.addTable(table.getProto());
+  }
+
+  @Test
+  public void testAddTable2() throws Exception {
+    TableDesc table = new TableDesc();
+
+    table.setName(DB_NAME + "." + REGION);
+
+    Options options = new Options();
+    options.put(HCatalogStore.CSVFILE_DELIMITER, "|");
+    options.put(HCatalogStore.CSVFILE_NULL, "\t");
+    TableMeta meta = new TableMeta(CatalogProtos.StoreType.CSV, options);
+    table.setMeta(meta);
+
+    table.setPath(new Path(warehousePath, REGION));
+
+    org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
+    schema.addColumn("r_regionkey", TajoDataTypes.Type.INT4);
+    schema.addColumn("r_name", TajoDataTypes.Type.TEXT);
+    schema.addColumn("r_comment", TajoDataTypes.Type.TEXT);
+
+    table.setSchema(schema);
+    store.addTable(table.getProto());
+  }
+
+  @Test
+  public void testAddTable3() throws Exception {
+    TableDesc table = new TableDesc();
+
+    table.setName(DB_NAME + "." + SUPPLIER);
+
+    Options options = new Options();
+    options.put(HCatalogStore.CSVFILE_DELIMITER, "\t");
+    options.put(HCatalogStore.CSVFILE_NULL, "\u0002");
+    TableMeta meta = new TableMeta(CatalogProtos.StoreType.CSV, options);
+    table.setMeta(meta);
+
+    table.setPath(new Path(warehousePath, SUPPLIER));
+
+    org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
+    schema.addColumn("s_suppkey", TajoDataTypes.Type.INT4);
+    schema.addColumn("s_name", TajoDataTypes.Type.TEXT);
+    schema.addColumn("s_address", TajoDataTypes.Type.TEXT);
+    schema.addColumn("s_nationkey", TajoDataTypes.Type.INT4);
+    schema.addColumn("s_phone", TajoDataTypes.Type.TEXT);
+    schema.addColumn("s_acctbal", TajoDataTypes.Type.FLOAT8);
+    schema.addColumn("s_comment", TajoDataTypes.Type.TEXT);
+
+    table.setSchema(schema);
+    store.addTable(table.getProto());
+  }
+
+  @Test
+  public void testAddTableByPartition() throws Exception {
+    TableDesc table = new TableDesc();
+
+    table.setName(DB_NAME + "." + NATION);
+
+    Options options = new Options();
+    options.put(HCatalogStore.CSVFILE_DELIMITER, "\u0001");
+    TableMeta meta = new TableMeta(CatalogProtos.StoreType.CSV, options);
+    table.setMeta(meta);
+
+    table.setPath(new Path(warehousePath, NATION));
+
+    org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
+    schema.addColumn("n_name", TajoDataTypes.Type.TEXT);
+    schema.addColumn("n_regionkey", TajoDataTypes.Type.INT4);
+    schema.addColumn("n_comment", TajoDataTypes.Type.TEXT);
+    table.setSchema(schema);
+
+    org.apache.tajo.catalog.Schema expressionSchema = new org.apache.tajo.catalog.Schema();
+    expressionSchema.addColumn("n_nationkey", TajoDataTypes.Type.INT4);
+
+    PartitionMethodDesc partitions = new PartitionMethodDesc(DB_NAME + "." + NATION,
+        CatalogProtos.PartitionType.COLUMN, expressionSchema.getColumn(0).getQualifiedName(), expressionSchema);
+    table.setPartitionMethod(partitions);
+
+    store.addTable(table.getProto());
+  }
 
   @Test
   public void testExistTable() throws Exception {
     assertTrue(store.existTable(DB_NAME + "." + CUSTOMER));
     assertTrue(store.existTable(DB_NAME + "." + NATION));
+    assertTrue(store.existTable(DB_NAME + "." + REGION));
+    assertTrue(store.existTable(DB_NAME + "." + SUPPLIER));
   }
 
   @Test
-  public void testGetTable() throws Exception {
+  public void testGetTable1() throws Exception {
     TableDesc table = new TableDesc(store.getTable(DB_NAME + "." + CUSTOMER));
 
     List<Column> columns = table.getSchema().getColumns();
@@ -269,37 +274,95 @@ public class TestHCatalogStore {
     assertEquals(TajoDataTypes.Type.TEXT, columns.get(6).getDataType().getType());
     assertEquals("c_comment", columns.get(7).getSimpleName());
     assertEquals(TajoDataTypes.Type.TEXT, columns.get(7).getDataType().getType());
+
     assertNull(table.getPartitionMethod());
+
     assertEquals(table.getMeta().getStoreType().name(), CatalogProtos.StoreType.RCFILE.name());
+  }
 
+  @Test
+  public void testGetTable2() throws Exception {
+    TableDesc table = new TableDesc(store.getTable(DB_NAME + "." + NATION));
 
-    table = new TableDesc(store.getTable(DB_NAME + "." + NATION));
-    columns = table.getSchema().getColumns();
+    List<Column> columns = table.getSchema().getColumns();
     assertEquals(DB_NAME + "." + NATION, table.getName());
-    assertEquals(4, columns.size());
-    assertEquals("n_nationkey", columns.get(0).getSimpleName());
-    assertEquals(TajoDataTypes.Type.INT4, columns.get(0).getDataType().getType());
-    assertEquals("n_name", columns.get(1).getSimpleName());
-    assertEquals(TajoDataTypes.Type.TEXT, columns.get(1).getDataType().getType());
-    assertEquals("n_regionkey", columns.get(2).getSimpleName());
-    assertEquals(TajoDataTypes.Type.INT4, columns.get(2).getDataType().getType());
-    assertEquals("n_comment", columns.get(3).getSimpleName());
-    assertEquals(TajoDataTypes.Type.TEXT, columns.get(3).getDataType().getType());
+    assertEquals(3, columns.size());
+    assertEquals("n_name", columns.get(0).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(0).getDataType().getType());
+    assertEquals("n_regionkey", columns.get(1).getSimpleName());
+    assertEquals(TajoDataTypes.Type.INT4, columns.get(1).getDataType().getType());
+    assertEquals("n_comment", columns.get(2).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(2).getDataType().getType());
+
     assertNotNull(table.getPartitionMethod());
-    assertEquals("type", table.getPartitionMethod().getExpressionSchema().getColumn(0).getSimpleName());
+
+    assertEquals("n_nationkey", table.getPartitionMethod().getExpressionSchema().getColumn(0).getSimpleName());
     assertEquals(CatalogProtos.PartitionType.COLUMN, table.getPartitionMethod().getPartitionType());
 
-    assertEquals(table.getMeta().getOption(HCatalogStore.CVSFILE_DELIMITER), "\\001");
     assertEquals(table.getMeta().getStoreType().name(), CatalogProtos.StoreType.CSV.name());
+    assertEquals(table.getMeta().getOption(HCatalogStore.CSVFILE_DELIMITER), StringEscapeUtils.escapeJava("\u0001"));
+    assertEquals(table.getMeta().getOption(HCatalogStore.CSVFILE_NULL), StringEscapeUtils.escapeJava("\\N"));
+  }
+
+  @Test
+  public void testGetTable3() throws Exception {
+    TableDesc table = new TableDesc(store.getTable(DB_NAME + "." + REGION));
+
+    List<Column> columns = table.getSchema().getColumns();
+    assertEquals(DB_NAME + "." + REGION, table.getName());
+    assertEquals(3, columns.size());
+    assertEquals("r_regionkey", columns.get(0).getSimpleName());
+    assertEquals(TajoDataTypes.Type.INT4, columns.get(0).getDataType().getType());
+    assertEquals("r_name", columns.get(1).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(1).getDataType().getType());
+    assertEquals("r_comment", columns.get(2).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(2).getDataType().getType());
+
+    assertNull(table.getPartitionMethod());
 
+    assertEquals(table.getMeta().getStoreType().name(), CatalogProtos.StoreType.CSV.name());
+    assertEquals(table.getMeta().getOption(HCatalogStore.CSVFILE_DELIMITER), StringEscapeUtils.escapeJava("|"));
+    assertEquals(table.getMeta().getOption(HCatalogStore.CSVFILE_NULL), StringEscapeUtils.escapeJava("\t"));
+  }
+
+  @Test
+  public void testGetTable4() throws Exception {
+    TableDesc table = new TableDesc(store.getTable(DB_NAME + "." + SUPPLIER));
+
+    List<Column> columns = table.getSchema().getColumns();
+    assertEquals(DB_NAME + "." + SUPPLIER, table.getName());
+    assertEquals(7, columns.size());
+    assertEquals("s_suppkey", columns.get(0).getSimpleName());
+    assertEquals(TajoDataTypes.Type.INT4, columns.get(0).getDataType().getType());
+    assertEquals("s_name", columns.get(1).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(1).getDataType().getType());
+    assertEquals("s_address", columns.get(2).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(2).getDataType().getType());
+    assertEquals("s_nationkey", columns.get(3).getSimpleName());
+    assertEquals(TajoDataTypes.Type.INT4, columns.get(3).getDataType().getType());
+    assertEquals("s_phone", columns.get(4).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(4).getDataType().getType());
+    assertEquals("s_acctbal", columns.get(5).getSimpleName());
+    assertEquals(TajoDataTypes.Type.FLOAT8, columns.get(5).getDataType().getType());
+    assertEquals("s_comment", columns.get(6).getSimpleName());
+    assertEquals(TajoDataTypes.Type.TEXT, columns.get(6).getDataType().getType());
+
+    assertNull(table.getPartitionMethod());
+
+    assertEquals(table.getMeta().getStoreType().name(), CatalogProtos.StoreType.CSV.name());
+    assertEquals(table.getMeta().getOption(HCatalogStore.CSVFILE_DELIMITER), StringEscapeUtils.escapeJava("\t"));
+    assertEquals(table.getMeta().getOption(HCatalogStore.CSVFILE_NULL), StringEscapeUtils.escapeJava("\u0002"));
   }
 
+
   @Test
   public void testGetAllTableNames() throws Exception{
     Set<String> tables = new HashSet<String>(store.getAllTableNames());
-    assertEquals(2, tables.size());
+    assertEquals(4, tables.size());
     assertTrue(tables.contains(DB_NAME + "." + CUSTOMER));
     assertTrue(tables.contains(DB_NAME + "." + NATION));
+    assertTrue(tables.contains(DB_NAME + "." + REGION));
+    assertTrue(tables.contains(DB_NAME + "." + SUPPLIER));
   }
 
   @Test
@@ -310,12 +373,22 @@ public class TestHCatalogStore {
     table = new TableDesc(store.getTable(DB_NAME + "." + NATION));
     Path nationPath = table.getPath();
 
+    table = new TableDesc(store.getTable(DB_NAME + "." + REGION));
+    Path regionPath = table.getPath();
+
+    table = new TableDesc(store.getTable(DB_NAME + "." + SUPPLIER));
+    Path supplierPath = table.getPath();
+
     store.deleteTable(DB_NAME + "." + CUSTOMER);
     store.deleteTable(DB_NAME + "." + NATION);
+    store.deleteTable(DB_NAME + "." + REGION);
+    store.deleteTable(DB_NAME + "." + SUPPLIER);
 
     FileSystem fs = FileSystem.getLocal(new Configuration());
     assertTrue(fs.exists(customerPath));
     assertTrue(fs.exists(nationPath));
+    assertTrue(fs.exists(regionPath));
+    assertTrue(fs.exists(supplierPath));
     fs.close();
   }
 }