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 2015/09/02 13:06:29 UTC

[06/39] tajo git commit: Implement the base.

Implement the base.


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

Branch: refs/heads/TAJO-1730
Commit: 39e7bd8e3f98565eae47eb8eae10888d655c11b9
Parents: 9b26ccc
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Aug 13 23:16:06 2015 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Aug 13 23:16:06 2015 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/tajo/util/UriUtil.java |  32 ++++++
 tajo-storage/tajo-storage-common/pom.xml        |   4 +-
 .../apache/tajo/storage/TablespaceManager.java  |  13 ++-
 .../src/main/resources/storage-default.json     |  12 +-
 .../tajo/storage/jdbc/ConnectionInfo.java       | 112 +++++++++++++++++++
 .../tajo/storage/jdbc/JdbcConnectionInfo.java   |  92 ---------------
 .../storage/jdbc/JdbcMetadataProviderBase.java  |   9 +-
 .../tajo/storage/jdbc/TestConnectionInfo.java   |  50 +++++++++
 .../storage/jdbc/TestJdbcConnectionInfo.java    |  50 ---------
 tajo-storage/tajo-storage-mysql/pom.xml         |  13 +--
 .../storage/jdbc/TestMysqlJdbcTableSpace.java   |  47 ++++----
 11 files changed, 245 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-common/src/main/java/org/apache/tajo/util/UriUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/UriUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/UriUtil.java
new file mode 100644
index 0000000..c0ee853
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/util/UriUtil.java
@@ -0,0 +1,32 @@
+/*
+ * 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.util;
+
+import java.net.URI;
+
+public class UriUtil {
+
+  public static String getScheme(URI uri) {
+    return getScheme(uri.toASCIIString());
+  }
+
+  public static String getScheme(String uri) {
+    return uri.substring(0, uri.indexOf(":/"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-common/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/pom.xml b/tajo-storage/tajo-storage-common/pom.xml
index f1d3438..ef0dc3a 100644
--- a/tajo-storage/tajo-storage-common/pom.xml
+++ b/tajo-storage/tajo-storage-common/pom.xml
@@ -50,8 +50,8 @@ limitations under the License.
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
           <encoding>${project.build.sourceEncoding}</encoding>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java
index 63cd502..ae3654d 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java
@@ -21,7 +21,6 @@ package org.apache.tajo.storage;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Maps;
@@ -37,6 +36,7 @@ import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.Pair;
+import org.apache.tajo.util.UriUtil;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
@@ -98,7 +98,7 @@ public class TablespaceManager implements StorageService {
   }
 
   private void addLocalFsTablespace() {
-    if (TABLE_SPACES.headMap(LOCAL_FS_URI, true).firstEntry() == null) {
+    if (TABLE_SPACES.headMap(LOCAL_FS_URI, true).firstEntry() == null && TABLE_SPACE_HANDLERS.containsKey("file")) {
       String tmpName = UUID.randomUUID().toString();
       registerTableSpace(tmpName, LOCAL_FS_URI, null, false, false);
     }
@@ -273,8 +273,7 @@ public class TablespaceManager implements StorageService {
   public static final String KEY_SPACES = "spaces";
 
   private static Tablespace initializeTableSpace(String spaceName, URI uri, JSONObject spaceDesc) {
-    Preconditions.checkNotNull(uri.getScheme(), "URI must include scheme, but it was " + uri);
-    Class<? extends Tablespace> clazz = TABLE_SPACE_HANDLERS.get(uri.getScheme());
+    Class<? extends Tablespace> clazz = TABLE_SPACE_HANDLERS.get(UriUtil.getScheme(uri));
 
     if (clazz == null) {
       throw new RuntimeException("There is no tablespace for " + uri.toString());
@@ -300,6 +299,12 @@ public class TablespaceManager implements StorageService {
   public static Optional<Tablespace> addTableSpaceForTest(Tablespace space) {
     Tablespace existing;
     synchronized (SPACES_URIS_MAP) {
+
+      String scheme = UriUtil.getScheme(space.getUri());
+      if (!TABLE_SPACE_HANDLERS.containsKey(scheme)) {
+        TABLE_SPACE_HANDLERS.put(scheme, space.getClass());
+      }
+
       // Remove existing one
       SPACES_URIS_MAP.remove(space.getName());
       existing = TABLE_SPACES.remove(space.getUri());

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json
index 16d20db..6060d50 100644
--- a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json
+++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json
@@ -12,13 +12,17 @@
       "handler": "org.apache.tajo.storage.FileTablespace",
       "default-format": "text"
     },
-    "hbase": {
+    "hbase:zk": {
       "handler": "org.apache.tajo.storage.hbase.HBaseTablespace",
       "default-format": "hbase"
     },
-    "jdbc": {
-      "handler": "org.apache.tajo.storage.jdbc.JdbcTablespace",
-      "default-format": "hbase"
+    "jdbc:mysql": {
+      "handler": "org.apache.tajo.storage.mysql.MySQLTablespace",
+      "default-format": "rowstore"
+    },
+    "jdbc:postgresql": {
+      "handler": "org.apache.tajo.storage.pgsql.PgSQLTablespace",
+      "default-format": "rowstore"
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/ConnectionInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/ConnectionInfo.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/ConnectionInfo.java
new file mode 100644
index 0000000..9a42b0d
--- /dev/null
+++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/ConnectionInfo.java
@@ -0,0 +1,112 @@
+/*
+ * 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.storage.jdbc;
+
+import org.apache.tajo.exception.TajoInternalError;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+
+public class ConnectionInfo {
+  String scheme;
+  String dbName;
+  String tableName;
+  String user;
+  String password;
+  Map<String, String> params;
+
+  public String scheme() {
+    return scheme;
+  }
+
+  public String database() {
+    return dbName;
+  }
+
+  public String table() {
+    return tableName;
+  }
+
+  public String user() {
+    return user;
+  }
+
+  public String password() {
+    return password;
+  }
+
+  public static ConnectionInfo fromURI(String originalUri) {
+    return fromURI(URI.create(originalUri));
+  }
+
+  public static ConnectionInfo fromURI(URI originalUri) {
+    final String uriStr = originalUri.toASCIIString();
+    URI uri = originalUri;
+
+    final ConnectionInfo connInfo = new ConnectionInfo();
+    connInfo.scheme = uriStr.substring(0, uriStr.indexOf("://"));
+
+    if (connInfo.scheme.split(":").length > 1) {
+      int idx = uriStr.indexOf(':');
+      uri = URI.create(uriStr.substring(idx + 1));
+    }
+
+    String path = uri.getPath();
+    if (path != null && !path.isEmpty()) {
+      String [] pathElements = path.substring(1).split("/");
+      if (pathElements.length != 1) {
+        throw new TajoInternalError("Invalid JDBC path: " + path);
+      }
+      connInfo.dbName = pathElements[0];
+    }
+
+    Map<String, String> params = new HashMap<>();
+
+    int paramIndex = uriStr.indexOf("?");
+    if (paramIndex > 0) {
+      String parameterPart = uriStr.substring(paramIndex+1, uriStr.length());
+
+      String [] eachParam = parameterPart.split("&");
+
+      for (String each: eachParam) {
+        String [] keyValues = each.split("=");
+        if (keyValues.length != 2) {
+          throw new TajoInternalError("Invalid URI Parameters: " + parameterPart);
+        }
+        params.put(keyValues[0], keyValues[1]);
+      }
+    }
+
+    if (params.containsKey("table")) {
+      connInfo.tableName = params.remove("table");
+    }
+
+    if (params.containsKey("user")) {
+      connInfo.user = params.remove("user");
+    }
+    if (params.containsKey("password")) {
+      connInfo.password = params.remove("password");
+    }
+
+    connInfo.params = params;
+
+    return connInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcConnectionInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcConnectionInfo.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcConnectionInfo.java
deleted file mode 100644
index 255aa84..0000000
--- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcConnectionInfo.java
+++ /dev/null
@@ -1,92 +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.storage.jdbc;
-
-import org.apache.tajo.exception.TajoInternalError;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
-
-public class JdbcConnectionInfo {
-  String scheme;
-  String dbName;
-  String tableName;
-  String user;
-  String password;
-  Map<String, String> params;
-
-  public static JdbcConnectionInfo fromURI(String originalUri) {
-    return fromURI(URI.create(originalUri));
-  }
-
-  public static JdbcConnectionInfo fromURI(URI originalUri) {
-    final String uriStr = originalUri.toASCIIString();
-    URI uri = originalUri;
-
-    final JdbcConnectionInfo connInfo = new JdbcConnectionInfo();
-    connInfo.scheme = uriStr.substring(0, uriStr.indexOf("://"));
-
-    if (connInfo.scheme.split(":").length > 1) {
-      int idx = uriStr.indexOf(':');
-      uri = URI.create(uriStr.substring(idx + 1));
-    }
-
-    String path = uri.getPath();
-    if (path != null && !path.isEmpty()) {
-      String [] pathElements = path.substring(1).split("/");
-      if (pathElements.length != 1) {
-        throw new TajoInternalError("Invalid JDBC path: " + path);
-      }
-      connInfo.dbName = pathElements[0];
-    }
-
-    Map<String, String> params = new HashMap<>();
-
-    int paramIndex = uriStr.indexOf("?");
-    if (paramIndex > 0) {
-      String parameterPart = uriStr.substring(paramIndex+1, uriStr.length());
-
-      String [] eachParam = parameterPart.split("&");
-
-      for (String each: eachParam) {
-        String [] keyValues = each.split("=");
-        if (keyValues.length != 2) {
-          throw new TajoInternalError("Invalid URI Parameters: " + parameterPart);
-        }
-        params.put(keyValues[0], keyValues[1]);
-      }
-    }
-
-    if (params.containsKey("table")) {
-      connInfo.tableName = params.remove("table");
-    }
-
-    if (params.containsKey("user")) {
-      connInfo.user = params.remove("user");
-    }
-    if (params.containsKey("password")) {
-      connInfo.password = params.remove("password");
-    }
-
-    connInfo.params = params;
-
-    return connInfo;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java
index 172b444..834643b 100644
--- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java
+++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java
@@ -57,15 +57,14 @@ public abstract class JdbcMetadataProviderBase implements MetadataProvider {
     this.space = space;
     this.databaseName = dbName;
 
-    JdbcConnectionInfo connInfo = JdbcConnectionInfo.fromURI(space.getUri());
+    ConnectionInfo connInfo = ConnectionInfo.fromURI(space.getUri());
     this.jdbcUri  = space.getUri().toASCIIString();
-    this.username = connInfo.user;
-    this.password = connInfo.password;
+    this.username = connInfo.user();
+    this.password = connInfo.password();
 
-//    String jdbcDriver = getJdbcDriverName();
     try {
       Class.forName(getJdbcDriverName()).newInstance();
-      LOG.info("Loaded JDBC driver (" + "com.mysql.jdbc.Driver" + ")");
+      LOG.info(getJdbcDriverName() + " is loaded...");
     } catch (Exception e) {
       throw new TajoInternalError(e);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestConnectionInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestConnectionInfo.java b/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestConnectionInfo.java
new file mode 100644
index 0000000..1ccd322
--- /dev/null
+++ b/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestConnectionInfo.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.jdbc;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestConnectionInfo {
+  @Test
+  public final void testGetConnectionInfoType1() {
+    ConnectionInfo c1 = ConnectionInfo.fromURI("jdbc:mysql://localhost:55840?user=testuser&password=testpass");
+    assertEquals("jdbc:mysql", c1.scheme);
+    assertEquals("testuser", c1.user);
+    assertEquals("testpass", c1.password);
+    assertNull(c1.dbName);
+    assertNull(c1.tableName);
+    assertEquals(0, c1.params.size());
+  }
+
+  @Test
+  public final void testGetConnectionInfoType2() {
+    ConnectionInfo c1 = ConnectionInfo.fromURI(
+        "jdbc:mysql://localhost:55840/db1?table=tb1&user=testuser&password=testpass&TZ=GMT+9");
+    assertEquals("jdbc:mysql", c1.scheme);
+    assertEquals("testuser", c1.user);
+    assertEquals("testpass", c1.password);
+    assertEquals("db1", c1.dbName);
+    assertEquals("tb1", c1.tableName);
+    assertEquals(1, c1.params.size());
+    assertEquals("GMT+9", c1.params.get("TZ"));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestJdbcConnectionInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestJdbcConnectionInfo.java b/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestJdbcConnectionInfo.java
deleted file mode 100644
index 9a340aa..0000000
--- a/tajo-storage/tajo-storage-jdbc/src/test/java/org/apache/tajo/storage/jdbc/TestJdbcConnectionInfo.java
+++ /dev/null
@@ -1,50 +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.storage.jdbc;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-public class TestJdbcConnectionInfo {
-  @Test
-  public final void testGetConnectionInfoType1() {
-    JdbcConnectionInfo c1 = JdbcConnectionInfo.fromURI("jdbc:mysql://localhost:55840?user=testuser&password=testpass");
-    assertEquals("jdbc:mysql", c1.scheme);
-    assertEquals("testuser", c1.user);
-    assertEquals("testpass", c1.password);
-    assertNull(c1.dbName);
-    assertNull(c1.tableName);
-    assertEquals(0, c1.params.size());
-  }
-
-  @Test
-  public final void testGetConnectionInfoType2() {
-    JdbcConnectionInfo c1 = JdbcConnectionInfo.fromURI(
-        "jdbc:mysql://localhost:55840/db1?table=tb1&user=testuser&password=testpass&TZ=GMT+9");
-    assertEquals("jdbc:mysql", c1.scheme);
-    assertEquals("testuser", c1.user);
-    assertEquals("testpass", c1.password);
-    assertEquals("db1", c1.dbName);
-    assertEquals("tb1", c1.tableName);
-    assertEquals(1, c1.params.size());
-    assertEquals("GMT+9", c1.params.get("TZ"));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-mysql/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-mysql/pom.xml b/tajo-storage/tajo-storage-mysql/pom.xml
index 3cca4f5..0294745 100644
--- a/tajo-storage/tajo-storage-mysql/pom.xml
+++ b/tajo-storage/tajo-storage-mysql/pom.xml
@@ -114,6 +114,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
       <artifactId>tajo-storage-jdbc</artifactId>
       <scope>provided</scope>
     </dependency>
@@ -236,14 +241,6 @@
       <version>0.1</version>
       <scope>test</scope>
     </dependency>
-    <!--
-    <dependency>
-      <groupId>io.airlift</groupId>
-      <artifactId>testing-postgresql-server</artifactId>
-      <version>0.3</version>
-      <scope>test</scope>
-    </dependency>
-    -->
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/39e7bd8e/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/jdbc/TestMysqlJdbcTableSpace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/jdbc/TestMysqlJdbcTableSpace.java b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/jdbc/TestMysqlJdbcTableSpace.java
index 0e3fb91..ed25d06 100644
--- a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/jdbc/TestMysqlJdbcTableSpace.java
+++ b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/jdbc/TestMysqlJdbcTableSpace.java
@@ -30,7 +30,6 @@ import org.apache.tajo.storage.mysql.MySQLTablespace;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.IOException;
 import java.net.URI;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -41,37 +40,37 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 public class TestMysqlJdbcTableSpace {
+
+  static TestingMySqlServer server;
+
   @BeforeClass
-  public static void setUp() throws IOException {
-    String mysqlUri = "jdbc:mysql://host1:2171/db1";
-    MySQLTablespace mysqlTablespace = new MySQLTablespace("cluster2", URI.create(mysqlUri), null);
+  public static void setUp() throws Exception {
+    server = new TestingMySqlServer("testuser", "testpass",
+        "meta_test",
+        "create_table",
+        "drop_table"
+    );
+
+    MySQLTablespace mysqlTablespace = new MySQLTablespace("mysql_cluster", URI.create(server.getJdbcUrl()), null);
     mysqlTablespace.init(new TajoConf());
     TablespaceManager.addTableSpaceForTest(mysqlTablespace);
   }
 
   @Test
   public void testTablespaceHandler() throws Exception {
-    assertTrue((TablespaceManager.getByName("cluster2").get()) instanceof MySQLTablespace);
-    assertEquals("cluster2", (TablespaceManager.getByName("cluster2").get().getName()));
-    assertTrue((TablespaceManager.get(URI.create("jdbc:mysql://host1:2171/db1")).get()) instanceof MySQLTablespace);
-    assertTrue((TablespaceManager.get(URI.create("jdbc:mysql://host1:2171/db1?table=xyz")).get())
-        instanceof MySQLTablespace);
-
-    assertEquals(URI.create("jdbc:mysql://host1:2171/db1"),
-        TablespaceManager.get(URI.create("jdbc:mysql://host1:2171/db1")).get().getUri());
-
-    assertTrue((TablespaceManager.getByName("cluster3").get()) instanceof MySQLTablespace);
-    assertEquals("cluster3", (TablespaceManager.getByName("cluster3").get().getName()));
-    assertTrue((TablespaceManager.get(URI.create("jdbc:postgres://host1:2615/db2")).get()) instanceof MySQLTablespace);
-    assertTrue((TablespaceManager.get(URI.create("jdbc:postgres://host1:2615/db2?table=xyz")).get())
-        instanceof MySQLTablespace);
-
-    assertEquals(URI.create("jdbc:postgres://host1:2615/db2"),
-        TablespaceManager.get(URI.create("jdbc:postgres://host1:2615/db2")).get().getUri());
+    assertTrue((TablespaceManager.getByName("mysql_cluster").get()) instanceof MySQLTablespace);
+    assertEquals("mysql_cluster", (TablespaceManager.getByName("mysql_cluster").get().getName()));
+
+    assertTrue((TablespaceManager.get(server.getJdbcUrl()).get()) instanceof MySQLTablespace);
+    assertTrue((TablespaceManager.get(server.getJdbcUrl() + "&table=tb1").get()) instanceof MySQLTablespace);
+
+    assertEquals(server.getJdbcUrl(), TablespaceManager.get(server.getJdbcUrl()).get().getUri().toASCIIString());
   }
 
   @Test
-  public void test() throws Exception {
+  public void testMetadataProvider() throws Exception {
+
+
     try (TestingMySqlServer server = new TestingMySqlServer("testuser", "testpass", "db1", "db2")) {
       assertTrue(server.isRunning());
       assertTrue(server.isReadyForConnections());
@@ -95,8 +94,8 @@ public class TestMysqlJdbcTableSpace {
       MySQLTablespace tablespace = new MySQLTablespace("mysql", URI.create(server.getJdbcUrl()), null);
 
       URI uri = tablespace.getTableUri("abc", "table1");
-      JdbcConnectionInfo c1 = JdbcConnectionInfo.fromURI(uri);
-      assertEquals("table1", c1.tableName);
+      ConnectionInfo c1 = ConnectionInfo.fromURI(uri);
+      assertEquals("table1", c1.table());
 
       MetadataProvider provider = tablespace.getMetadataProvider();
       Set<String> tables = Sets.newHashSet(provider.getTables(null, null));