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 2014/10/17 02:24:39 UTC

git commit: TAJO-233: Support PostgreSQL CatalogStore. (Jihun Kang via hyunsik)

Repository: tajo
Updated Branches:
  refs/heads/master d56737b99 -> 2e32f5a62


TAJO-233: Support PostgreSQL CatalogStore. (Jihun Kang via hyunsik)


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

Branch: refs/heads/master
Commit: 2e32f5a6271d6952f77839ff639e11537af03ebf
Parents: d56737b
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Oct 16 17:09:15 2014 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Oct 16 17:09:15 2014 -0700

----------------------------------------------------------------------
 CHANGES                                         |   1 +
 .../tajo/catalog/store/PostgreSQLStore.java     | 332 +++++++++++++++++++
 .../resources/schemas/postgresql/columns.sql    |   9 +
 .../resources/schemas/postgresql/databases.sql  |   6 +
 .../resources/schemas/postgresql/indexes.sql    |  14 +
 .../schemas/postgresql/partition_methods.sql    |   7 +
 .../resources/schemas/postgresql/partitions.sql |  10 +
 .../main/resources/schemas/postgresql/stats.sql |   6 +
 .../schemas/postgresql/table_properties.sql     |   7 +
 .../resources/schemas/postgresql/tables.sql     |   9 +
 .../schemas/postgresql/tablespaces.sql          |   6 +
 .../org/apache/tajo/catalog/TestCatalog.java    |  11 +-
 12 files changed, 416 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 85f587a..3a306a5 100644
--- a/CHANGES
+++ b/CHANGES
@@ -5,6 +5,7 @@ Release 0.9.1 - unreleased
 
   NEW FEATURES
 
+    TAJO-233: Support PostgreSQL CatalogStore. (Jihun Kang via hyunsik)
 
   IMPROVEMENT
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java
new file mode 100644
index 0000000..a06cf19
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java
@@ -0,0 +1,332 @@
+/**
+ * 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 java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.exception.CatalogException;
+import org.apache.tajo.exception.InternalException;
+
+public class PostgreSQLStore extends AbstractDBStore {
+  
+  private static final int POSTGRESQL_STORE_VERSION = 2;
+  private static final String CATALOG_DRIVER = "org.postgresql.Driver";
+
+  public PostgreSQLStore(Configuration conf) throws InternalException {
+    super(conf);
+  }
+
+  @Override
+  protected String getCatalogDriverName() {
+    return CATALOG_DRIVER;
+  }
+
+  @Override
+  protected Connection createConnection(Configuration conf) throws SQLException {
+    return DriverManager.getConnection(getCatalogUri(), this.connectionId, this.connectionPassword);
+  }
+
+  @Override
+  protected boolean isInitialized() throws CatalogException {
+    Connection conn;
+    ResultSet res = null;
+
+    try {
+      conn = getConnection();
+      res = conn.getMetaData().getTables(null, null, null, new String[] { "TABLE" });
+      
+      baseTableMaps.put(TB_META, false);
+      baseTableMaps.put(TB_SPACES, false);
+      baseTableMaps.put(TB_DATABASES, false);
+      baseTableMaps.put(TB_TABLES, false);
+      baseTableMaps.put(TB_COLUMNS, false);
+      baseTableMaps.put(TB_OPTIONS, false);
+      baseTableMaps.put(TB_STATISTICS, false);
+      baseTableMaps.put(TB_INDEXES, false);
+      baseTableMaps.put(TB_PARTITION_METHODS, false);
+      baseTableMaps.put(TB_PARTTIONS, false);
+      
+      while (res.next()) {
+        baseTableMaps.put(res.getString("TABLE_NAME").toUpperCase(), true);
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(res);
+    }
+    
+    for(Map.Entry<String, Boolean> entry : baseTableMaps.entrySet()) {
+      if (!entry.getValue()) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  protected void createBaseTable() throws CatalogException {
+    Statement stmt = null;
+    Connection conn = null;
+
+    try {
+      conn = getConnection();
+      stmt = conn.createStatement();
+
+
+      // META
+      if (!baseTableMaps.get(TB_META)) {
+        String sql = super.readSchemaFile("common/meta.sql");
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+
+        stmt.executeUpdate(sql.toString());
+        LOG.info("Table '" + TB_META + " is created.");
+        baseTableMaps.put(TB_META, true);
+      }
+
+      // TABLE SPACES
+      if (!baseTableMaps.get(TB_SPACES)) {
+        String sql = readSchemaFile("tablespaces.sql");
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        sql = "CREATE UNIQUE INDEX TABLESPACES_IDX_NAME on TABLESPACES (SPACE_NAME)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_SPACES + "' is created.");
+        baseTableMaps.put(TB_SPACES, true);
+      }
+
+      // DATABASES
+      if (!baseTableMaps.get(TB_DATABASES)) {
+        String sql = readSchemaFile("databases.sql");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }        
+        stmt.addBatch(sql);
+        
+        sql = "CREATE UNIQUE INDEX DATABASES__IDX_NAME on DATABASES_ (DB_NAME)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_DATABASES + "' is created.");
+        baseTableMaps.put(TB_DATABASES, true);
+      }
+
+      // TABLES
+      if (!baseTableMaps.get(TB_TABLES)) {
+        String sql = readSchemaFile("tables.sql");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        sql = "CREATE INDEX TABLES_IDX_DB_ID on TABLES (DB_ID)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        sql = "CREATE UNIQUE INDEX TABLES_IDX_TABLE_ID on TABLES (DB_ID, TABLE_NAME)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_TABLES + "' is created.");
+        baseTableMaps.put(TB_TABLES, true);
+      }
+
+      // COLUMNS
+      if (!baseTableMaps.get(TB_COLUMNS)) {
+        String sql = readSchemaFile("columns.sql");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+
+        stmt.executeUpdate(sql);
+        LOG.info("Table '" + TB_COLUMNS + " is created.");
+        baseTableMaps.put(TB_COLUMNS, true);
+      }
+
+      // OPTIONS
+      if (!baseTableMaps.get(TB_OPTIONS)) {
+        String sql = readSchemaFile("table_properties.sql");
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+
+        stmt.executeUpdate(sql);
+        LOG.info("Table '" + TB_OPTIONS + " is created.");
+        baseTableMaps.put(TB_OPTIONS, true);
+      }
+
+      // INDEXES
+      if (!baseTableMaps.get(TB_INDEXES)) {
+        String sql = readSchemaFile("indexes.sql");
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql.toString());
+        
+        sql = "CREATE UNIQUE INDEX INDEXES_IDX_DB_ID_NAME on INDEXES (DB_ID, INDEX_NAME)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        sql = "CREATE INDEX INDEXES_IDX_TID_COLUMN_NAME on INDEXES (TID, COLUMN_NAME)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_INDEXES + "' is created.");
+        baseTableMaps.put(TB_INDEXES, true);
+      }
+
+      if (!baseTableMaps.get(TB_STATISTICS)) {
+        String sql = readSchemaFile("stats.sql");
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql.toString());
+        }
+
+        stmt.executeUpdate(sql.toString());
+        LOG.info("Table '" + TB_STATISTICS + "' is created.");
+        baseTableMaps.put(TB_STATISTICS, true);
+      }
+
+      // PARTITION_METHODS
+      if (!baseTableMaps.get(TB_PARTITION_METHODS)) {
+        String sql = readSchemaFile("partition_methods.sql");
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+
+        stmt.executeUpdate(sql);
+        LOG.info("Table '" + TB_PARTITION_METHODS + "' is created.");
+        baseTableMaps.put(TB_PARTITION_METHODS, true);
+      }
+
+      // PARTITIONS
+      if (!baseTableMaps.get(TB_PARTTIONS)) {
+        String sql = readSchemaFile("partitions.sql");
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        sql = "CREATE INDEX PARTITIONS_IDX_TID on PARTITIONS (TID)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        sql = "CREATE UNIQUE INDEX IDX_TID_NAME on PARTITIONS (TID, PARTITION_NAME)";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
+        stmt.addBatch(sql);
+        
+        stmt.executeBatch();
+        LOG.info("Table '" + TB_PARTTIONS + "' is created.");
+        baseTableMaps.put(TB_PARTTIONS, true);
+      }
+
+      insertSchemaVersion();
+
+    } catch (SQLException se) {
+      throw new CatalogException("failed to create base tables for PostgreSQL catalog store", se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt);
+    }
+  }
+
+  @Override
+  protected void dropBaseTable() throws CatalogException {
+    Connection conn;
+    Statement stmt = null;
+    Map<String, Boolean> droppedTable = new HashMap<String, Boolean>();
+
+    try {
+      conn = getConnection();
+      stmt = conn.createStatement();
+
+      for(Map.Entry<String, Boolean> entry : baseTableMaps.entrySet()) {
+        if(entry.getValue() && !entry.getKey().equals(TB_TABLES)) {
+          String sql = "DROP TABLE " + entry.getKey();
+          stmt.addBatch(sql);
+          droppedTable.put(entry.getKey(), true);
+        }
+      }
+      if(baseTableMaps.get(TB_TABLES)) {
+        String sql = "DROP TABLE " + TB_TABLES;
+        stmt.addBatch(sql);
+        droppedTable.put(TB_TABLES, true);
+      }
+      stmt.executeBatch();
+
+      for(String tableName : droppedTable.keySet()) {
+        LOG.info("Table '" + tableName + "' is dropped");
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt);
+    }
+  }
+
+  @Override
+  public int getDriverVersion() {
+    return POSTGRESQL_STORE_VERSION;
+  }
+
+  @Override
+  public String readSchemaFile(String path) throws CatalogException {
+    return super.readSchemaFile("postgresql/" + path);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/columns.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/columns.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/columns.sql
new file mode 100644
index 0000000..425be96
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/columns.sql
@@ -0,0 +1,9 @@
+CREATE TABLE COLUMNS (
+  TID INT NOT NULL,
+  COLUMN_NAME VARCHAR(255) NOT NULL,
+  ORDINAL_POSITION INT NOT NULL,
+  DATA_TYPE CHAR(16),
+  TYPE_LENGTH INTEGER,
+  CONSTRAINT COLUMNS_PKEY PRIMARY KEY (TID, COLUMN_NAME),
+  FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/databases.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/databases.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/databases.sql
new file mode 100644
index 0000000..8232884
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/databases.sql
@@ -0,0 +1,6 @@
+CREATE TABLE DATABASES_ (
+  DB_ID SERIAL NOT NULL PRIMARY KEY,
+  DB_NAME VARCHAR(128) NOT NULL UNIQUE,
+  SPACE_ID INT NOT NULL,
+  FOREIGN KEY (SPACE_ID) REFERENCES TABLESPACES (SPACE_ID)
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/indexes.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/indexes.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/indexes.sql
new file mode 100644
index 0000000..cbb28fc
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/indexes.sql
@@ -0,0 +1,14 @@
+CREATE TABLE INDEXES (
+  DB_ID INT NOT NULL,
+  TID INT NOT NULL,
+  INDEX_NAME VARCHAR(128) NOT NULL,
+  COLUMN_NAME VARCHAR(128) NOT NULL,
+  DATA_TYPE VARCHAR(128) NOT NULL,
+  INDEX_TYPE CHAR(32) NOT NULL,
+  IS_UNIQUE BOOLEAN NOT NULL,
+  IS_CLUSTERED BOOLEAN NOT NULL,
+  IS_ASCENDING BOOLEAN NOT NULL,
+  CONSTRAINT INDEXES_PKEY PRIMARY KEY (DB_ID, INDEX_NAME),
+  FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID) ON DELETE CASCADE,
+  FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partition_methods.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partition_methods.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partition_methods.sql
new file mode 100644
index 0000000..55289ae
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partition_methods.sql
@@ -0,0 +1,7 @@
+CREATE TABLE PARTITION_METHODS (
+  TID INT NOT NULL PRIMARY KEY,
+  PARTITION_TYPE VARCHAR(10) NOT NULL,
+  EXPRESSION VARCHAR(1024) NOT NULL,
+  EXPRESSION_SCHEMA BYTEA NOT NULL,
+  FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partitions.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partitions.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partitions.sql
new file mode 100644
index 0000000..d46a59a
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/partitions.sql
@@ -0,0 +1,10 @@
+CREATE TABLE PARTITIONS (
+  PID INT NOT NULL PRIMARY KEY,
+  TID INT NOT NULL,
+  PARTITION_NAME VARCHAR(128),
+  ORDINAL_POSITION INT NOT NULL,
+  PARTITION_VALUE VARCHAR(1024),
+  PATH VARCHAR(4096),
+  FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE,
+  CONSTRAINT C_PARTITION_UNIQUE UNIQUE (TID, PARTITION_NAME)
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/stats.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/stats.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/stats.sql
new file mode 100644
index 0000000..bba8ee7
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/stats.sql
@@ -0,0 +1,6 @@
+CREATE TABLE STATS (
+  TID INT NOT NULL PRIMARY KEY,
+  NUM_ROWS BIGINT,
+  NUM_BYTES BIGINT,
+  FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/table_properties.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/table_properties.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/table_properties.sql
new file mode 100644
index 0000000..57aa316
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/table_properties.sql
@@ -0,0 +1,7 @@
+CREATE TABLE OPTIONS (
+  TID INT NOT NULL,
+  KEY_ VARCHAR(255) NOT NULL,
+  VALUE_ VARCHAR(255) NOT NULL,
+  CONSTRAINT OPTIONS_PKEY PRIMARY KEY (TID, KEY_),
+  FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tables.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tables.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tables.sql
new file mode 100644
index 0000000..d87264a
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tables.sql
@@ -0,0 +1,9 @@
+CREATE TABLE TABLES (
+  TID SERIAL NOT NULL PRIMARY KEY,
+  DB_ID INT NOT NULL,
+  TABLE_NAME VARCHAR(128) NOT NULL,
+  TABLE_TYPE VARCHAR(128) NOT NULL,
+  PATH VARCHAR(4096),
+  STORE_TYPE CHAR(16),
+  FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID)
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tablespaces.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tablespaces.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tablespaces.sql
new file mode 100644
index 0000000..65a366d
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/tablespaces.sql
@@ -0,0 +1,6 @@
+CREATE TABLE TABLESPACES (
+  SPACE_ID SERIAL NOT NULL PRIMARY KEY,
+  SPACE_NAME VARCHAR(128) NOT NULL UNIQUE,
+  SPACE_HANDLER VARCHAR (1024) DEFAULT 'HDFS',
+  SPACE_URI VARCHAR (4096) NOT NULL
+)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/2e32f5a6/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
index aa3f1c2..f586660 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.exception.CatalogException;
 import org.apache.tajo.catalog.exception.NoSuchFunctionException;
+import org.apache.tajo.catalog.store.PostgreSQLStore;
 import org.apache.tajo.function.Function;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -86,8 +87,8 @@ public class TestCatalog {
     conf.set(CATALOG_URI, catalogURI);
     conf.setVar(TajoConf.ConfVars.CATALOG_ADDRESS, "127.0.0.1:0");
 
-    // MySQLStore/MariaDB requires password
-    if (driverClass.equals(MySQLStore.class.getCanonicalName()) || driverClass.equals(MariaDBStore.class.getCanonicalName())) {
+    // MySQLStore/MariaDB/PostgreSQL requires username (and password).
+    if (isConnectionIdRequired(driverClass)) {
       if (connectionId == null) {
         throw new CatalogException(String.format("%s driver requires %s", driverClass, CatalogConstants.CONNECTION_ID));
       }
@@ -114,6 +115,12 @@ public class TestCatalog {
       catalog.dropTable(table);
     }
 	}
+
+  public static boolean isConnectionIdRequired(String driverClass) {
+    return driverClass.equals(MySQLStore.class.getCanonicalName()) ||
+           driverClass.equals(MariaDBStore.class.getCanonicalName()) ||
+           driverClass.equals(PostgreSQLStore.class.getCanonicalName());
+  }
 	
 	@AfterClass
 	public static void tearDown() throws IOException {