You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hj...@apache.org on 2014/12/05 09:21:12 UTC

[08/29] tajo git commit: TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)

TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)


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

Branch: refs/heads/hbase_storage
Commit: 7d41c67ca94493e38c67e62663dc097358a18539
Parents: 4637f7f
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Nov 27 23:19:31 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Nov 27 23:19:31 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/client/ResultSetUtil.java   |   8 +
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java | 181 ++++++++++++++-----
 .../dataset/TestTajoJdbc/table1/table1.tbl      |   5 +
 .../TestTajoJdbc/create_table_with_date_ddl.sql |  10 +
 .../TestTajoJdbc/testSortWithDateTime.result    |   7 +
 .../org/apache/tajo/jdbc/MetaDataTuple.java     |   4 +-
 7 files changed, 174 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 8bf0c1c..0d0677c 100644
--- a/CHANGES
+++ b/CHANGES
@@ -77,6 +77,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)
+
     TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)
 
     TAJO-1208: Failure of create table using textfile on hivemeta.

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java b/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
index 056eb2c..9211a1b 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
@@ -75,6 +75,10 @@ public class ResultSetUtil {
       return "character";
     case DATE:
       return "date";
+    case TIMESTAMP:
+      return "timestamp";
+    case TIME:
+      return "time";
     case VARCHAR:
       return "varchar";
     case TEXT:
@@ -103,7 +107,11 @@ public class ResultSetUtil {
     case NUMERIC:
       return Types.NUMERIC;
     case DATE:
+      return Types.DATE;
+    case TIMESTAMP:
       return Types.TIMESTAMP;
+    case TIME:
+      return Types.TIME;
     case VARCHAR:
       return Types.VARCHAR;
     case TEXT:

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
index 08535ef..a004baa 100644
--- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
+++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
@@ -26,6 +26,7 @@ import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryClient;
+import org.apache.tajo.conf.TajoConf;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -33,13 +34,11 @@ import org.junit.experimental.categories.Category;
 
 import java.net.InetSocketAddress;
 import java.sql.*;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
 import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 @Category(IntegrationTest.class)
 public class TestTajoJdbc extends QueryTestCaseBase {
@@ -69,13 +68,13 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test(expected = SQLException.class)
   public void testGetConnection() throws SQLException {
     DriverManager.getConnection("jdbc:taju://" + tajoMasterAddress.getHostName() + ":" + tajoMasterAddress.getPort()
-        + "/default");
+      + "/default");
   }
 
   @Test
   public void testStatement() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -85,10 +84,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       stmt = conn.createStatement();
 
       res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-          "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
+        "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
 
       try {
-        Map<String,Integer> result = Maps.newHashMap();
+        Map<String, Integer> result = Maps.newHashMap();
         result.put("NO", 3);
         result.put("RF", 2);
 
@@ -108,10 +107,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
         res.close();
       }
     } finally {
-      if(res != null) {
+      if (res != null) {
         res.close();
       }
-      if(stmt != null) {
+      if (stmt != null) {
         stmt.close();
       }
     }
@@ -120,7 +119,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testPreparedStatement() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -137,7 +136,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       */
 
       String sql =
-          "select l_orderkey, l_quantity, l_returnflag from lineitem where l_quantity > ? and l_returnflag = ?";
+        "select l_orderkey, l_quantity, l_returnflag from lineitem where l_quantity > ? and l_returnflag = ?";
 
       stmt = conn.prepareStatement(sql);
 
@@ -155,9 +154,9 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       try {
         int numRows = 0;
         String[] resultData = {"136.0N", "238.0N"};
-        while(res.next()) {
+        while (res.next()) {
           assertEquals(resultData[numRows],
-              ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
+            ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
           numRows++;
         }
         assertEquals(2, numRows);
@@ -179,9 +178,9 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       try {
         int numRows = 0;
         String[] resultData = {"345.0R", "349.0R"};
-        while(res.next()) {
+        while (res.next()) {
           assertEquals(resultData[numRows],
-              ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
+            ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
           numRows++;
         }
         assertEquals(2, numRows);
@@ -189,10 +188,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
         res.close();
       }
     } finally {
-      if(res != null) {
+      if (res != null) {
         res.close();
       }
-      if(stmt != null) {
+      if (stmt != null) {
         stmt.close();
       }
     }
@@ -201,7 +200,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testDatabaseMetaDataGetTable() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -219,12 +218,12 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       Set<String> retrivedViaJavaAPI = new HashSet<String>(client.getTableList("default"));
 
       Set<String> retrievedViaJDBC = new HashSet<String>();
-      while(rs.next()) {
+      while (rs.next()) {
         retrievedViaJDBC.add(rs.getString("TABLE_NAME"));
       }
       assertEquals(retrievedViaJDBC, retrivedViaJavaAPI);
     } finally {
-      if(rs != null) {
+      if (rs != null) {
         rs.close();
       }
     }
@@ -236,7 +235,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testDatabaseMetaDataGetColumns() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -258,7 +257,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
 
       List<Column> columns = tableDesc.getSchema().getColumns();
 
-      while(rs.next()) {
+      while (rs.next()) {
         assertEquals(tableName, rs.getString("TABLE_NAME"));
         assertEquals(columns.get(numColumns).getSimpleName(), rs.getString("COLUMN_NAME"));
         // TODO assert type
@@ -267,7 +266,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
 
       assertEquals(16, numColumns);
     } finally {
-      if(rs != null) {
+      if (rs != null) {
         rs.close();
       }
     }
@@ -280,24 +279,24 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testMultipleConnections() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
 
     Connection[] conns = new Connection[2];
     conns[0] = DriverManager.getConnection(connUri);
     conns[1] = DriverManager.getConnection(connUri);
 
     try {
-      for(int i = 0; i < conns.length; i++) {
+      for (int i = 0; i < conns.length; i++) {
         Statement stmt = null;
         ResultSet res = null;
         try {
           stmt = conns[i].createStatement();
 
           res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-              "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
+            "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
 
           try {
-            Map<String,Integer> result = Maps.newHashMap();
+            Map<String, Integer> result = Maps.newHashMap();
             result.put("NO", 3);
             result.put("RF", 2);
 
@@ -317,10 +316,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
             res.close();
           }
         } finally {
-          if(res != null) {
+          if (res != null) {
             res.close();
           }
-          if(stmt != null) {
+          if (stmt != null) {
             stmt.close();
           }
         }
@@ -338,24 +337,24 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testMultipleConnectionsSequentialClose() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
 
     Connection[] conns = new Connection[2];
     conns[0] = DriverManager.getConnection(connUri);
     conns[1] = DriverManager.getConnection(connUri);
 
     try {
-      for(int i = 0; i < conns.length; i++) {
+      for (int i = 0; i < conns.length; i++) {
         Statement stmt = null;
         ResultSet res = null;
         try {
           stmt = conns[i].createStatement();
 
           res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-              "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
+            "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
 
           try {
-            Map<String,Integer> result = Maps.newHashMap();
+            Map<String, Integer> result = Maps.newHashMap();
             result.put("NO", 3);
             result.put("RF", 2);
 
@@ -375,22 +374,22 @@ public class TestTajoJdbc extends QueryTestCaseBase {
             res.close();
           }
         } finally {
-          if(res != null) {
+          if (res != null) {
             res.close();
           }
-          if(stmt != null) {
+          if (stmt != null) {
             stmt.close();
           }
           conns[i].close();
         }
       }
     } finally {
-      if(!conns[0].isClosed()) {
+      if (!conns[0].isClosed()) {
         assertTrue(conns[0].isValid(100));
         conns[0].close();
         assertFalse(conns[0].isValid(100));
       }
-      if(!conns[1].isClosed()) {
+      if (!conns[1].isClosed()) {
         assertTrue(conns[1].isValid(100));
         conns[1].close();
         assertFalse(conns[1].isValid(100));
@@ -405,7 +404,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
     assertFalse(TajoStatement.isSetVariableQuery("--SET JOIN_TASK_INPUT_SIZE 123"));
 
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
 
     Connection conn = DriverManager.getConnection(connUri);
 
@@ -419,7 +418,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       assertNotNull(rsmd);
       assertEquals(0, rsmd.getColumnCount());
 
-      QueryClient connTajoClient = ((JdbcConnection)stmt.getConnection()).getQueryClient();
+      QueryClient connTajoClient = ((JdbcConnection) stmt.getConnection()).getQueryClient();
       Map<String, String> variables = connTajoClient.getAllSessionVariables();
       String value = variables.get("JOIN_TASK_INPUT_SIZE");
       assertNotNull(value);
@@ -447,7 +446,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testSetPreparedStatement() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
 
     Connection conn = DriverManager.getConnection(connUri);
 
@@ -461,7 +460,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       assertNotNull(rsmd);
       assertEquals(0, rsmd.getColumnCount());
 
-      QueryClient connTajoClient = ((JdbcConnection)stmt.getConnection()).getQueryClient();
+      QueryClient connTajoClient = ((JdbcConnection) stmt.getConnection()).getQueryClient();
       Map<String, String> variables = connTajoClient.getAllSessionVariables();
       String value = variables.get("JOIN_TASK_INPUT_SIZE");
       assertNotNull(value);
@@ -487,4 +486,102 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       }
     }
   }
+
+  @Test
+  public void testCreateTableWithDateAndTimestamp() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testCreateTableWithDateAndTimestamp");
+
+    int result;
+    Statement stmt = null;
+    ResultSet res = null;
+
+    try {
+      String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
+        DEFAULT_DATABASE_NAME);
+      Connection conn = DriverManager.getConnection(connUri);
+      assertTrue(conn.isValid(100));
+
+      stmt = conn.createStatement();
+      result = stmt.executeUpdate("create table " + tableName + " (id int, name text, score double"
+        + ", register_date timestamp, update_date date, send_date time)");
+      assertEquals(result, 1);
+
+      res = stmt.executeQuery("select * from " + tableName);
+      assertFalse(res.next());
+
+      ResultSetMetaData rsmd = res.getMetaData();
+      assertNotNull(rsmd);
+      assertEquals(6, rsmd.getColumnCount());
+
+      assertEquals("id", rsmd.getColumnName(1));
+      assertEquals("name", rsmd.getColumnName(2));
+      assertEquals("score", rsmd.getColumnName(3));
+      assertEquals("register_date", rsmd.getColumnName(4));
+      assertEquals("update_date", rsmd.getColumnName(5));
+      assertEquals("send_date", rsmd.getColumnName(6));
+
+      assertEquals("integer", rsmd.getColumnTypeName(1));
+      assertEquals("varchar", rsmd.getColumnTypeName(2));
+      assertEquals("float8", rsmd.getColumnTypeName(3));
+      assertEquals("timestamp", rsmd.getColumnTypeName(4));
+      assertEquals("date", rsmd.getColumnTypeName(5));
+      assertEquals("time", rsmd.getColumnTypeName(6));
+
+    } finally {
+      cleanupQuery(res);
+      if (stmt != null) {
+        stmt.close();
+      }
+    }
+  }
+
+  @Test
+  public void testSortWithDateTime() throws Exception {
+    Statement stmt = null;
+    ResultSet res = null;
+    int result;
+
+    // skip this test if catalog uses HCatalogStore.
+    // It is because HCatalogStore does not support Time data type.
+    TimeZone oldTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("UTC"));
+    TimeZone systemOldTimeZone = TimeZone.getDefault();
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+
+    try {
+      if (!testingCluster.isHCatalogStoreRunning()) {
+        executeDDL("create_table_with_date_ddl.sql", "table1");
+
+        String connUri = buildConnectionUri(tajoMasterAddress.getHostName(),
+          tajoMasterAddress.getPort(), "TestTajoJdbc");
+
+        Connection conn = DriverManager.getConnection(connUri);
+        assertTrue(conn.isValid(100));
+
+        stmt = conn.createStatement();
+        res = stmt.executeQuery("select col1, col2, col3 from table1 order by col1, col2, col3");
+
+        ResultSetMetaData rsmd = res.getMetaData();
+        assertNotNull(rsmd);
+        assertEquals(3, rsmd.getColumnCount());
+
+        assertEquals("timestamp", rsmd.getColumnTypeName(1));
+        assertEquals("date", rsmd.getColumnTypeName(2));
+        assertEquals("time", rsmd.getColumnTypeName(3));
+
+        assertResultSet(res);
+
+        result = stmt.executeUpdate("drop table table1");
+        assertEquals(result, 1);
+
+      }
+    } finally {
+      TajoConf.setCurrentTimeZone(oldTimeZone);
+      TimeZone.setDefault(systemOldTimeZone);
+
+      cleanupQuery(res);
+      if (stmt != null) {
+        stmt.close();
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl b/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
new file mode 100644
index 0000000..52fa2fe
--- /dev/null
+++ b/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
@@ -0,0 +1,5 @@
+1997-11-09 20:34:56|1996-04-12|15:34:56
+1997-11-09 20:34:56|1996-03-13|19:34:56
+1993-11-09 20:34:56|1997-01-28|08:34:56
+1995-11-09 20:34:56|1994-02-02|17:34:56
+1995-11-09 20:34:56|1993-11-09|20:34:56

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql b/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
new file mode 100644
index 0000000..846cbb6
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
@@ -0,0 +1,10 @@
+-- Sort Table
+-- It is used in TestSortQuery::testSortWithDate
+
+create external table table1 (
+  col1 timestamp,
+	col2 date,
+	col3 time
+) using csv
+with ('csvfile.delimiter'='|', 'csvfile.null'='NULL')
+location ${table.path};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result b/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result
new file mode 100644
index 0000000..118909c
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result
@@ -0,0 +1,7 @@
+col1,col2,col3
+-------------------------------
+1993-11-09 20:34:56,1997-01-28,08:34:56
+1995-11-09 20:34:56,1993-11-09,20:34:56
+1995-11-09 20:34:56,1994-02-02,17:34:56
+1997-11-09 20:34:56,1996-03-13,19:34:56
+1997-11-09 20:34:56,1996-04-12,15:34:56
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
----------------------------------------------------------------------
diff --git a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
index 6c8ef5d..30fc9ee 100644
--- a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
+++ b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
@@ -1,4 +1,4 @@
-package org.apache.tajo.jdbc; /**
+/**
  * 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
@@ -16,6 +16,8 @@ package org.apache.tajo.jdbc; /**
  * limitations under the License.
  */
 
+package org.apache.tajo.jdbc;
+
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.IntervalDatum;
 import org.apache.tajo.datum.NullDatum;