You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2015/03/18 18:25:40 UTC

[01/13] tajo git commit: TAJO-1398: Fix 'Getting Started' link error from 0.9.0 into 0.10.0. (Contributed by Dongjoon Hyun, Committed by jihoon)

Repository: tajo
Updated Branches:
  refs/heads/index_support 63c8e1c00 -> 60cbe9cac


TAJO-1398: Fix 'Getting Started' link error from 0.9.0 into 0.10.0. (Contributed by Dongjoon Hyun, Committed by jihoon)

Closes #419


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

Branch: refs/heads/index_support
Commit: d8db8b776113ad67a8e52829cebb920793b7fca9
Parents: f48d4bd
Author: Jihoon Son <ji...@apache.org>
Authored: Fri Mar 13 13:30:59 2015 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Fri Mar 13 13:31:18 2015 +0900

----------------------------------------------------------------------
 CHANGES                        | 3 +++
 tajo-project/src/site/site.xml | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/d8db8b77/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 9bec78c..4eee74f 100644
--- a/CHANGES
+++ b/CHANGES
@@ -48,6 +48,9 @@ Release 0.11.0 - unreleased
   
   TASKS
 
+    TAJO-1398: Fix 'Getting Started' link error from 0.9.0 into 0.10.0. 
+    (Contributed by Dongjoon Hyun, Committed by jihoon)
+
     TAJO-1380: Update JDBC documentation for new JDBC driver.
     (Contributed by Dongjoon Hyun, Committed by hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d8db8b77/tajo-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/tajo-project/src/site/site.xml b/tajo-project/src/site/site.xml
index fd3bb6f..a679e33 100644
--- a/tajo-project/src/site/site.xml
+++ b/tajo-project/src/site/site.xml
@@ -109,7 +109,7 @@ window.onload = ga_event_load;
       <item name="Overview" href="index.html" />
       <item name="License" href="license.html" />
       <item name="Downloads" href="downloads.html" />
-      <item name="Getting Started" href="docs/0.9.0/getting_started.html" />
+      <item name="Getting Started" href="docs/current/getting_started.html" />
     </menu>
 
     <menu name="Community">


[12/13] tajo git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into index_support

Posted by ji...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into index_support

Conflicts:
	tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/indexes.sql


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

Branch: refs/heads/index_support
Commit: 6b30cfe627c5413349d87c6104ee3461b90ef727
Parents: 292cd9e a9ae3ca
Author: Jihoon Son <ji...@apache.org>
Authored: Thu Mar 19 02:25:03 2015 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Thu Mar 19 02:25:03 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |  24 ++
 .../apache/tajo/algebra/ColumnDefinition.java   |   2 +-
 .../org/apache/tajo/algebra/DataTypeExpr.java   |  18 +-
 tajo-catalog/tajo-catalog-client/pom.xml        |   2 +-
 .../org/apache/tajo/catalog/CatalogUtil.java    |  28 +-
 .../java/org/apache/tajo/catalog/Column.java    |  45 ++-
 .../org/apache/tajo/catalog/DDLBuilder.java     |   9 +-
 .../java/org/apache/tajo/catalog/Schema.java    | 156 ++++++---
 .../org/apache/tajo/catalog/SchemaUtil.java     |  42 +++
 .../java/org/apache/tajo/catalog/TypeDesc.java  |  75 ++++
 .../org/apache/tajo/catalog/TestSchema.java     | 137 ++++++++
 .../tajo-catalog-drivers/tajo-hcatalog/pom.xml  |   2 +-
 tajo-catalog/tajo-catalog-server/pom.xml        |   2 +-
 .../dictionary/ColumnsTableDescriptor.java      |   7 +-
 .../tajo/catalog/store/AbstractDBStore.java     |  64 +++-
 .../apache/tajo/catalog/store/MariaDBStore.java |   5 +-
 .../apache/tajo/catalog/store/MySQLStore.java   |  11 +-
 .../apache/tajo/catalog/store/OracleStore.java  |   2 +-
 .../tajo/catalog/store/PostgreSQLStore.java     |   4 +-
 .../src/main/resources/schemas/derby/derby.xml  |  12 +-
 .../main/resources/schemas/mariadb/columns.sql  |   3 +-
 .../main/resources/schemas/mysql/columns.sql    |   3 +-
 .../main/resources/schemas/oracle/columns.sql   |   9 -
 .../main/resources/schemas/oracle/databases.sql |   6 -
 .../main/resources/schemas/oracle/indexes.sql   |  18 -
 .../main/resources/schemas/oracle/oracle.xml    |  10 +-
 .../schemas/oracle/partition_methods.sql        |   7 -
 .../resources/schemas/oracle/partitions.sql     |  10 -
 .../src/main/resources/schemas/oracle/stats.sql |   6 -
 .../schemas/oracle/table_properties.sql         |   7 -
 .../main/resources/schemas/oracle/tables.sql    |   9 -
 .../resources/schemas/oracle/tablespaces.sql    |   6 -
 .../resources/schemas/postgresql/columns.sql    |   9 -
 .../resources/schemas/postgresql/databases.sql  |   6 -
 .../schemas/postgresql/partition_methods.sql    |   7 -
 .../resources/schemas/postgresql/partitions.sql |  10 -
 .../resources/schemas/postgresql/postgresql.xml |  10 +-
 .../schemas/postgresql/table_properties.sql     |   7 -
 .../resources/schemas/postgresql/tables.sql     |   9 -
 .../schemas/postgresql/tablespaces.sql          |   6 -
 .../org/apache/tajo/catalog/TestCatalog.java    | 101 ++++++
 tajo-cli/pom.xml                                |   2 +-
 .../cli/tsql/commands/DescTableCommand.java     |   5 +-
 tajo-client/pom.xml                             |   2 +-
 .../apache/tajo/client/SessionConnection.java   |  20 +-
 .../java/org/apache/tajo/conf/TajoConf.java     |   2 +
 .../java/org/apache/tajo/util/BytesUtils.java   | 159 +++++----
 .../java/org/apache/tajo/util/StringUtils.java  |   6 +-
 tajo-common/src/main/proto/DataTypes.proto      |  10 +
 .../org/apache/tajo/util/TestStringUtil.java    |   4 +-
 tajo-core/pom.xml                               |  58 ++-
 .../engine/planner/physical/HashJoinExec.java   |  51 ++-
 .../planner/physical/HashLeftOuterJoinExec.java |  74 +++-
 .../physical/PartitionMergeScanExec.java        |  17 +-
 .../tajo/engine/planner/physical/ScanExec.java  |  72 ++++
 .../engine/planner/physical/SeqScanExec.java    | 105 +-----
 .../apache/tajo/engine/utils/CacheHolder.java   |  97 ++++++
 .../apache/tajo/engine/utils/TableCache.java    |  84 +++++
 .../apache/tajo/engine/utils/TableCacheKey.java |  57 +++
 .../apache/tajo/engine/utils/TupleCache.java    | 122 -------
 .../apache/tajo/engine/utils/TupleCacheKey.java |  57 ---
 .../tajo/worker/ExecutionBlockContext.java      |  43 ++-
 .../worker/ExecutionBlockSharedResource.java    |  26 ++
 .../main/java/org/apache/tajo/worker/Task.java  |  75 ++--
 .../apache/tajo/worker/TaskAttemptContext.java  |   2 +-
 .../java/org/apache/tajo/worker/TaskRunner.java |   9 +-
 .../apache/tajo/worker/TaskRunnerManager.java   |   4 +-
 .../org/apache/tajo/cli/tools/TestTajoDump.java |  21 ++
 .../org/apache/tajo/cli/tsql/TestTajoCli.java   |  40 ++-
 .../apache/tajo/engine/eval/ExprTestBase.java   |   5 +-
 .../tajo/engine/query/TestCreateTable.java      |  24 ++
 .../tajo/engine/query/TestSelectQuery.java      |  48 +++
 .../apache/tajo/engine/util/TestTableCache.java | 109 ++++++
 .../apache/tajo/engine/util/TestTupleCache.java |  89 -----
 .../multibytes_delimiter1/table1.tbl            |   5 +
 .../multibytes_delimiter2/table2.tbl            |   5 +
 .../TestCreateTable/testNestedRecord1.sql       |   1 +
 .../TestCreateTable/testNestedRecord2.sql       |   1 +
 .../multibytes_delimiter_table1_ddl.sql         |   3 +
 .../multibytes_delimiter_table2_ddl.sql         |   3 +
 .../multibytes_delimiter_table3_ddl.sql         |   3 +
 .../multibytes_delimiter_table4_ddl.sql         |   3 +
 .../testMultiBytesDelimiter1.sql                |   1 +
 .../testMultiBytesDelimiter2.sql                |   1 +
 .../testMultiBytesDelimiter3.sql                |   1 +
 .../testMultiBytesDelimiter4.sql                |   1 +
 .../TestSelectQuery/testExplainSelect.result    |   4 +-
 .../testMultiBytesDelimiter1.result             |   7 +
 .../testMultiBytesDelimiter2.result             |   7 +
 .../testMultiBytesDelimiter3.result             |   7 +
 .../testMultiBytesDelimiter4.result             |   7 +
 .../testDescTableForNestedSchema.result         |  29 ++
 .../results/TestTajoDump/testDump2.result       |  16 +
 tajo-dist/pom.xml                               |   7 +-
 .../org/apache/tajo/plan/ExprAnnotator.java     |   5 +-
 .../org/apache/tajo/plan/LogicalPlanner.java    |  18 +-
 .../org/apache/tajo/plan/TypeDeterminant.java   |   4 +-
 .../plan/serder/LogicalNodeDeserializer.java    |   3 +
 .../tajo/plan/serder/LogicalNodeSerializer.java |   2 +
 tajo-plan/src/main/proto/Plan.proto             |   1 +
 tajo-project/pom.xml                            |  31 ++
 tajo-project/src/site/site.xml                  |   2 +-
 tajo-pullserver/pom.xml                         |   2 +-
 tajo-rpc/pom.xml                                | 180 ++--------
 .../org/apache/tajo/rpc/AsyncRpcClient.java     | 267 --------------
 .../org/apache/tajo/rpc/AsyncRpcServer.java     | 148 --------
 .../org/apache/tajo/rpc/BlockingRpcClient.java  | 315 -----------------
 .../org/apache/tajo/rpc/BlockingRpcServer.java  | 147 --------
 .../java/org/apache/tajo/rpc/CallFuture.java    |  84 -----
 .../apache/tajo/rpc/DefaultRpcController.java   |  65 ----
 .../org/apache/tajo/rpc/NettyClientBase.java    | 173 ---------
 .../org/apache/tajo/rpc/NettyRpcController.java |  63 ----
 .../org/apache/tajo/rpc/NettyServerBase.java    | 205 -----------
 .../java/org/apache/tajo/rpc/NullCallback.java  |  38 --
 .../tajo/rpc/ProtoChannelInitializer.java       |  50 ---
 .../apache/tajo/rpc/RemoteCallException.java    |  69 ----
 .../org/apache/tajo/rpc/RemoteException.java    |  37 --
 .../tajo/rpc/RetriesExhaustedException.java     | 104 ------
 .../org/apache/tajo/rpc/RpcChannelFactory.java  | 182 ----------
 .../org/apache/tajo/rpc/RpcConnectionPool.java  | 194 -----------
 .../main/java/org/apache/tajo/rpc/RpcUtils.java |  68 ----
 .../org/apache/tajo/rpc/ServerCallable.java     | 162 ---------
 .../apache/tajo/rpc/TajoServiceException.java   |  58 ---
 tajo-rpc/src/main/proto/DummyProtos.proto       |  47 ---
 tajo-rpc/src/main/proto/RpcProtos.proto         |  32 --
 tajo-rpc/src/main/proto/TestProtocol.proto      |  31 --
 tajo-rpc/src/main/proto/TestProtos.proto        |  35 --
 tajo-rpc/src/test/java/log4j.properties         |  25 --
 .../java/org/apache/tajo/rpc/TestAsyncRpc.java  | 339 ------------------
 .../org/apache/tajo/rpc/TestBlockingRpc.java    | 341 ------------------
 .../rpc/test/impl/DummyProtocolAsyncImpl.java   |  86 -----
 .../test/impl/DummyProtocolBlockingImpl.java    |  83 -----
 tajo-rpc/tajo-rpc-common/pom.xml                | 216 ++++++++++++
 .../org/apache/tajo/rpc/NettyServerBase.java    | 243 +++++++++++++
 .../org/apache/tajo/rpc/RemoteException.java    |  37 ++
 .../tajo/rpc/RetriesExhaustedException.java     | 104 ++++++
 .../org/apache/tajo/rpc/RpcChannelFactory.java  | 182 ++++++++++
 .../org/apache/tajo/rpc/RpcEventListener.java   |  62 ++++
 .../main/java/org/apache/tajo/rpc/RpcUtils.java | 122 +++++++
 tajo-rpc/tajo-rpc-protobuf/pom.xml              | 274 +++++++++++++++
 .../org/apache/tajo/rpc/AsyncRpcClient.java     | 227 ++++++++++++
 .../org/apache/tajo/rpc/AsyncRpcServer.java     | 148 ++++++++
 .../org/apache/tajo/rpc/BlockingRpcClient.java  | 273 +++++++++++++++
 .../org/apache/tajo/rpc/BlockingRpcServer.java  | 147 ++++++++
 .../java/org/apache/tajo/rpc/CallFuture.java    |  84 +++++
 .../apache/tajo/rpc/DefaultRpcController.java   |  65 ++++
 .../org/apache/tajo/rpc/NettyClientBase.java    | 221 ++++++++++++
 .../org/apache/tajo/rpc/NettyRpcController.java |  63 ++++
 .../java/org/apache/tajo/rpc/NullCallback.java  |  38 ++
 .../tajo/rpc/ProtoChannelInitializer.java       |  50 +++
 .../apache/tajo/rpc/RemoteCallException.java    |  69 ++++
 .../org/apache/tajo/rpc/RemoteException.java    |  37 ++
 .../tajo/rpc/RetriesExhaustedException.java     | 104 ++++++
 .../org/apache/tajo/rpc/RpcConnectionPool.java  | 190 ++++++++++
 .../org/apache/tajo/rpc/ServerCallable.java     | 162 +++++++++
 .../apache/tajo/rpc/TajoServiceException.java   |  58 +++
 .../src/main/proto/DummyProtos.proto            |  47 +++
 .../src/main/proto/RpcProtos.proto              |  32 ++
 .../src/main/proto/TestProtocol.proto           |  31 ++
 .../src/main/proto/TestProtos.proto             |  35 ++
 .../src/test/java/log4j.properties              |  25 ++
 .../java/org/apache/tajo/rpc/TestAsyncRpc.java  | 345 ++++++++++++++++++
 .../org/apache/tajo/rpc/TestBlockingRpc.java    | 349 +++++++++++++++++++
 .../rpc/test/impl/DummyProtocolAsyncImpl.java   |  86 +++++
 .../test/impl/DummyProtocolBlockingImpl.java    |  83 +++++
 tajo-rpc/tajo-ws-rs/pom.xml                     | 218 ++++++++++++
 .../rs/netty/NettyRestChannelInitializer.java   |  50 +++
 .../ws/rs/netty/NettyRestHandlerContainer.java  | 319 +++++++++++++++++
 .../NettyRestHandlerContainerProvider.java      |  42 +++
 .../tajo/ws/rs/netty/NettyRestServer.java       |  67 ++++
 .../ws/rs/netty/NettyRestServerFactory.java     |  89 +++++
 .../ws/rs/netty/NettyRestServerListener.java    |  72 ++++
 .../tajo/ws/rs/netty/gson/GsonFeature.java      |  34 ++
 .../tajo/ws/rs/netty/gson/GsonReader.java       |  52 +++
 .../apache/tajo/ws/rs/netty/gson/GsonUtil.java  |  32 ++
 .../tajo/ws/rs/netty/gson/GsonWriter.java       |  59 ++++
 .../NettyRestHandlerContainerProviderTest.java  |  66 ++++
 .../tajo/ws/rs/netty/NettyRestServerTest.java   | 137 ++++++++
 .../ws/rs/netty/testapp1/TestApplication1.java  |  38 ++
 .../ws/rs/netty/testapp1/TestResource1.java     |  36 ++
 .../ws/rs/netty/testapp2/DirectoriesDao.java    |  39 +++
 .../rs/netty/testapp2/DirectoriesResource.java  |  85 +++++
 .../tajo/ws/rs/netty/testapp2/Directory.java    |  52 +++
 .../testapp2/FileManagementApplication.java     |  35 ++
 .../org/apache/tajo/storage/TestLazyTuple.java  |   4 +-
 .../tajo/storage/hbase/ColumnMapping.java       |   6 +-
 .../apache/tajo/storage/hbase/HBaseScanner.java |   3 +-
 .../tajo/storage/hbase/HBaseStorageManager.java |   3 +-
 .../java/org/apache/tajo/storage/CSVFile.java   |  21 +-
 .../sequencefile/SequenceFileScanner.java       |   3 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |  14 +-
 .../apache/tajo/storage/text/CSVLineSerDe.java  |   5 +-
 .../tajo/storage/text/CSVLineSerializer.java    |   8 +-
 .../tajo/storage/text/DelimitedTextFile.java    |   2 +-
 .../tajo/storage/text/FieldSplitProcessor.java  |   8 +-
 .../text/MultiBytesFieldSplitProcessor.java     |  45 +++
 .../tajo/storage/text/TextLineDeserializer.java |   6 +-
 .../apache/tajo/storage/text/TextLineSerDe.java |   3 +-
 .../apache/tajo/storage/TestSplitProcessor.java |  38 +-
 199 files changed, 7679 insertions(+), 4434 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/6b30cfe6/tajo-plan/src/main/proto/Plan.proto
----------------------------------------------------------------------


[05/13] tajo git commit: TAJO-1405: Fix some illegal way of usages on connection pool. (Contributed by navis, Committed by Keuntae Park)

Posted by ji...@apache.org.
TAJO-1405: Fix some illegal way of usages on connection pool. (Contributed by navis, Committed by Keuntae Park)

Closes #425


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

Branch: refs/heads/index_support
Commit: 286b956795a4dc2efb72c97896d86ed1049485e3
Parents: 0dc7d68
Author: Keuntae Park <si...@apache.org>
Authored: Tue Mar 17 13:47:20 2015 +0900
Committer: Keuntae Park <si...@apache.org>
Committed: Tue Mar 17 13:47:20 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |  3 +
 .../apache/tajo/client/SessionConnection.java   | 18 +++--
 .../tajo/worker/ExecutionBlockContext.java      | 43 +++++++----
 .../main/java/org/apache/tajo/worker/Task.java  | 75 +++++++++++---------
 .../java/org/apache/tajo/worker/TaskRunner.java |  9 ++-
 .../org/apache/tajo/rpc/AsyncRpcClient.java     |  9 ++-
 .../org/apache/tajo/rpc/NettyClientBase.java    |  6 ++
 7 files changed, 103 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/286b9567/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 9d2cd14..0d7222f 100644
--- a/CHANGES
+++ b/CHANGES
@@ -29,6 +29,9 @@ Release 0.11.0 - unreleased
 
   BUG FIXES
 
+    TAJO-1405: Fix some illegal way of usages on connection pool. 
+    (Contributed by navis, Committed by Keuntae Park)
+
     TAJO-1384: Duplicated output file path problem. (jihoon)
 
     TAJO-1386: CURRENT_DATE generates parsing errors sometimes.

http://git-wip-us.apache.org/repos/asf/tajo/blob/286b9567/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
index d24e7b3..c084d95 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
@@ -57,7 +57,7 @@ public class SessionConnection implements Closeable {
 
   final RpcConnectionPool connPool;
 
-  private final String baseDatabase;
+  private String baseDatabase;
 
   private final UserRoleInfo userInfo;
 
@@ -260,7 +260,8 @@ public class SessionConnection implements Closeable {
   }
 
   public Boolean selectDatabase(final String databaseName) throws ServiceException {
-    return new ServerCallable<Boolean>(connPool, getTajoMasterAddr(), TajoMasterClientProtocol.class, false, true) {
+    Boolean selected = new ServerCallable<Boolean>(connPool, getTajoMasterAddr(),
+        TajoMasterClientProtocol.class, false, true) {
 
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
@@ -269,6 +270,11 @@ public class SessionConnection implements Closeable {
         return tajoMasterService.selectDatabase(null, convertSessionedString(databaseName)).getValue();
       }
     }.withRetries();
+
+    if (selected == Boolean.TRUE) {
+      this.baseDatabase = databaseName;
+    }
+    return selected;
   }
 
   @Override
@@ -278,13 +284,15 @@ public class SessionConnection implements Closeable {
     }
 
     // remove session
+    NettyClientBase client = null;
     try {
-
-      NettyClientBase client = connPool.getConnection(getTajoMasterAddr(), TajoMasterClientProtocol.class, false);
+      client = connPool.getConnection(getTajoMasterAddr(), TajoMasterClientProtocol.class, false);
       TajoMasterClientProtocolService.BlockingInterface tajoMaster = client.getStub();
       tajoMaster.removeSession(null, sessionId);
-
     } catch (Throwable e) {
+      // ignore
+    } finally {
+      connPool.releaseConnection(client);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/286b9567/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
index a645689..2377720 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
@@ -36,13 +36,11 @@ import org.apache.tajo.ipc.QueryMasterProtocol;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
 import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.NullCallback;
-import org.apache.tajo.rpc.RpcChannelFactory;
 import org.apache.tajo.rpc.RpcConnectionPool;
 import org.apache.tajo.storage.HashShuffleAppenderManager;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.util.Pair;
-import org.apache.tajo.worker.event.TaskRunnerStartEvent;
 
 import io.netty.channel.ConnectTimeoutException;
 import io.netty.channel.EventLoopGroup;
@@ -139,7 +137,17 @@ public class ExecutionBlockContext {
     try{
       this.resource.initialize(queryContext, plan);
     } catch (Throwable e) {
-      getQueryMasterStub().killQuery(null, executionBlockId.getQueryId().getProto(), NullCallback.get());
+      try {
+        NettyClientBase client = getQueryMasterConnection();
+        try {
+          QueryMasterProtocol.QueryMasterProtocolService.Interface stub = client.getStub();
+          stub.killQuery(null, executionBlockId.getQueryId().getProto(), NullCallback.get());
+        } finally {
+          connPool.releaseConnection(client);
+        }
+      } catch (Throwable t) {
+        //ignore
+      }
       throw e;
     }
   }
@@ -148,15 +156,13 @@ public class ExecutionBlockContext {
     return resource;
   }
 
-  public QueryMasterProtocol.QueryMasterProtocolService.Interface getQueryMasterStub()
+  public NettyClientBase getQueryMasterConnection()
       throws NoSuchMethodException, ConnectTimeoutException, ClassNotFoundException {
-    NettyClientBase clientBase = null;
-    try {
-      clientBase = connPool.getConnection(qmMasterAddr, QueryMasterProtocol.class, true);
-      return clientBase.getStub();
-    } finally {
-      connPool.releaseConnection(clientBase);
-    }
+    return connPool.getConnection(qmMasterAddr, QueryMasterProtocol.class, true);
+  }
+
+  public void releaseConnection(NettyClientBase connection) {
+    connPool.releaseConnection(connection);
   }
 
   public void stop(){
@@ -267,7 +273,13 @@ public class ExecutionBlockContext {
   }
 
   private void sendExecutionBlockReport(ExecutionBlockReport reporter) throws Exception {
-    getQueryMasterStub().doneExecutionBlock(null, reporter, NullCallback.get());
+    NettyClientBase client = getQueryMasterConnection();
+    try {
+      QueryMasterProtocol.QueryMasterProtocolService.Interface stub = client.getStub();
+      stub.doneExecutionBlock(null, reporter, NullCallback.get());
+    } finally {
+      connPool.releaseConnection(client);
+    }
   }
 
   protected void reportExecutionBlock(ExecutionBlockId ebId) {
@@ -361,12 +373,14 @@ public class ExecutionBlockContext {
 
       return new Runnable() {
         int remainingRetries = MAX_RETRIES;
-        QueryMasterProtocol.QueryMasterProtocolService.Interface masterStub;
         @Override
         public void run() {
           while (!reporterStop.get() && !Thread.interrupted()) {
+
+            NettyClientBase client = null;
             try {
-              masterStub = getQueryMasterStub();
+              client = getQueryMasterConnection();
+              QueryMasterProtocol.QueryMasterProtocolService.Interface masterStub = client.getStub();
 
               if(tasks.size() == 0){
                 masterStub.ping(null, getExecutionBlockId().getProto(), NullCallback.get());
@@ -391,6 +405,7 @@ public class ExecutionBlockContext {
                 throw new RuntimeException(t);
               }
             } finally {
+              releaseConnection(client);
               if (remainingRetries > 0 && !reporterStop.get()) {
                 synchronized (reporterThread) {
                   try {

http://git-wip-us.apache.org/repos/asf/tajo/blob/286b9567/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
index 524b09b..9ff18dd 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
@@ -52,6 +52,7 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.pullserver.TajoPullServerService;
 import org.apache.tajo.pullserver.retriever.FileChunk;
+import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.NullCallback;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
@@ -424,46 +425,52 @@ public class Task {
 
       executionBlockContext.completedTasksNum.incrementAndGet();
       context.getHashShuffleAppenderManager().finalizeTask(taskId);
-      QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getQueryMasterStub();
-      if (context.isStopped()) {
-        context.setExecutorProgress(0.0f);
 
-        if(context.getState() == TaskAttemptState.TA_KILLED) {
-          queryMasterStub.statusUpdate(null, getReport(), NullCallback.get());
-          executionBlockContext.killedTasksNum.incrementAndGet();
-        } else {
-          context.setState(TaskAttemptState.TA_FAILED);
-          TaskFatalErrorReport.Builder errorBuilder =
-              TaskFatalErrorReport.newBuilder()
-                  .setId(getId().getProto());
-          if (error != null) {
-            if (error.getMessage() == null) {
-              errorBuilder.setErrorMessage(error.getClass().getCanonicalName());
-            } else {
-              errorBuilder.setErrorMessage(error.getMessage());
+      NettyClientBase client = executionBlockContext.getQueryMasterConnection();
+      try {
+        QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = client.getStub();
+        if (context.isStopped()) {
+          context.setExecutorProgress(0.0f);
+
+          if (context.getState() == TaskAttemptState.TA_KILLED) {
+            queryMasterStub.statusUpdate(null, getReport(), NullCallback.get());
+            executionBlockContext.killedTasksNum.incrementAndGet();
+          } else {
+            context.setState(TaskAttemptState.TA_FAILED);
+            TaskFatalErrorReport.Builder errorBuilder =
+                TaskFatalErrorReport.newBuilder()
+                    .setId(getId().getProto());
+            if (error != null) {
+              if (error.getMessage() == null) {
+                errorBuilder.setErrorMessage(error.getClass().getCanonicalName());
+              } else {
+                errorBuilder.setErrorMessage(error.getMessage());
+              }
+              errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error));
             }
-            errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error));
+
+            queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get());
+            executionBlockContext.failedTasksNum.incrementAndGet();
           }
+        } else {
+          // if successful
+          context.setProgress(1.0f);
+          context.setState(TaskAttemptState.TA_SUCCEEDED);
+          executionBlockContext.succeededTasksNum.incrementAndGet();
 
-          queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get());
-          executionBlockContext.failedTasksNum.incrementAndGet();
+          TaskCompletionReport report = getTaskCompletionReport();
+          queryMasterStub.done(null, report, NullCallback.get());
         }
-      } else {
-        // if successful
-        context.setProgress(1.0f);
-        context.setState(TaskAttemptState.TA_SUCCEEDED);
-        executionBlockContext.succeededTasksNum.incrementAndGet();
-
-        TaskCompletionReport report = getTaskCompletionReport();
-        queryMasterStub.done(null, report, NullCallback.get());
+        finishTime = System.currentTimeMillis();
+        LOG.info(context.getTaskId() + " completed. " +
+            "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() +
+            ", succeeded: " + executionBlockContext.succeededTasksNum.intValue()
+            + ", killed: " + executionBlockContext.killedTasksNum.intValue()
+            + ", failed: " + executionBlockContext.failedTasksNum.intValue());
+        cleanupTask();
+      } finally {
+        executionBlockContext.releaseConnection(client);
       }
-      finishTime = System.currentTimeMillis();
-      LOG.info(context.getTaskId() + " completed. " +
-          "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() +
-          ", succeeded: " + executionBlockContext.succeededTasksNum.intValue()
-          + ", killed: " + executionBlockContext.killedTasksNum.intValue()
-          + ", failed: " + executionBlockContext.failedTasksNum.intValue());
-      cleanupTask();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/286b9567/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
index 79725f6..058ea42 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
@@ -35,6 +35,7 @@ import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.master.container.TajoContainerIdPBImpl;
 import org.apache.tajo.master.container.TajoConverterUtils;
 import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.NullCallback;
 
 import io.netty.channel.ConnectTimeoutException;
@@ -196,9 +197,9 @@ public class TaskRunner extends AbstractService {
           TaskRequestProto taskRequest = null;
 
           while(!stopped) {
-            QueryMasterProtocolService.Interface qmClientService;
+            NettyClientBase client;
             try {
-              qmClientService = getContext().getQueryMasterStub();
+              client = executionBlockContext.getQueryMasterConnection();
             } catch (ConnectTimeoutException ce) {
               // NettyClientBase throws ConnectTimeoutException if connection was failed
               stop();
@@ -212,6 +213,8 @@ public class TaskRunner extends AbstractService {
               break;
             }
 
+            QueryMasterProtocolService.Interface qmClientService = client.getStub();
+
             try {
               if (callFuture == null) {
                 callFuture = new CallFuture<TaskRequestProto>();
@@ -296,6 +299,8 @@ public class TaskRunner extends AbstractService {
               }
             } catch (Throwable t) {
               LOG.fatal(t.getMessage(), t);
+            } finally {
+              executionBlockContext.releaseConnection(client);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/tajo/blob/286b9567/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
index 1ea9fb1..3d856ce 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
@@ -35,11 +35,12 @@ import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 public class AsyncRpcClient extends NettyClientBase {
   private static final Log LOG = LogFactory.getLog(AsyncRpcClient.class);
 
-  private final Map<Integer, ResponseCallback> requests =
+  private final ConcurrentMap<Integer, ResponseCallback> requests =
       new ConcurrentHashMap<Integer, ResponseCallback>();
 
   private final Method stubMethod;
@@ -178,14 +179,12 @@ public class AsyncRpcClient extends NettyClientBase {
   @ChannelHandler.Sharable
   private class ClientChannelInboundHandler extends ChannelInboundHandlerAdapter {
 
-    synchronized void registerCallback(int seqId, ResponseCallback callback) {
+    void registerCallback(int seqId, ResponseCallback callback) {
 
-      if (requests.containsKey(seqId)) {
+      if (requests.putIfAbsent(seqId, callback) != null) {
         throw new RemoteException(
             getErrorMessage("Duplicate Sequence Id "+ seqId));
       }
-
-      requests.put(seqId, callback);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/286b9567/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
index 7dfc5a2..72278f2 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
@@ -132,6 +132,12 @@ public abstract class NettyClientBase implements Closeable {
     final CountDownLatch ticket = new CountDownLatch(1);
     final CountDownLatch granted = connect.check(ticket);
 
+    // basically, it's double checked lock
+    if (ticket == granted && isConnected()) {
+      granted.countDown();
+      return true;
+    }
+
     if (ticket == granted) {
       connectUsingNetty(addr, new RetryConnectionListener(addr, granted));
     }


[02/13] tajo git commit: TAJO-1374: Support multi-bytes delimiter for CSV file

Posted by ji...@apache.org.
TAJO-1374: Support multi-bytes delimiter for CSV file

closes #400

Signed-off-by: Jinho Kim <jh...@apache.org>


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

Branch: refs/heads/index_support
Commit: 7f0569555332665141e71141d7a736371d038509
Parents: d8db8b7
Author: navis.ryu <na...@apache.org>
Authored: Fri Mar 6 14:19:15 2015 +0900
Committer: Jinho Kim <jh...@apache.org>
Committed: Fri Mar 13 15:33:08 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../java/org/apache/tajo/util/BytesUtils.java   | 159 +++++++++++--------
 .../java/org/apache/tajo/util/StringUtils.java  |   6 +-
 .../org/apache/tajo/util/TestStringUtil.java    |   4 +-
 .../apache/tajo/engine/eval/ExprTestBase.java   |   5 +-
 .../tajo/engine/query/TestSelectQuery.java      |  24 +++
 .../multibytes_delimiter1/table1.tbl            |   5 +
 .../multibytes_delimiter2/table2.tbl            |   5 +
 .../multibytes_delimiter_table1_ddl.sql         |   3 +
 .../multibytes_delimiter_table2_ddl.sql         |   3 +
 .../testMultiBytesDelimiter1.sql                |   1 +
 .../testMultiBytesDelimiter2.sql                |   1 +
 .../testMultiBytesDelimiter1.result             |   7 +
 .../testMultiBytesDelimiter2.result             |   7 +
 .../org/apache/tajo/storage/TestLazyTuple.java  |   4 +-
 .../tajo/storage/hbase/ColumnMapping.java       |   6 +-
 .../apache/tajo/storage/hbase/HBaseScanner.java |   3 +-
 .../tajo/storage/hbase/HBaseStorageManager.java |   3 +-
 .../java/org/apache/tajo/storage/CSVFile.java   |  14 +-
 .../sequencefile/SequenceFileScanner.java       |   3 +-
 20 files changed, 180 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 4eee74f..e6f7917 100644
--- a/CHANGES
+++ b/CHANGES
@@ -9,6 +9,9 @@ Release 0.11.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1374: Support multi-bytes delimiter for CSV file.
+    (Contributed by navis, Committed by jinho)
+
     TAJO-1395: Remove deprecated sql files for Oracle and PostgreSQL. (jihun)
 
     TAJO-1394: Support reconnect on tsql. 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-common/src/main/java/org/apache/tajo/util/BytesUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/BytesUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/BytesUtils.java
index 91165ac..725301c 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/BytesUtils.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/BytesUtils.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.io.WritableUtils;
 
 import java.io.ByteArrayOutputStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 /**
@@ -86,22 +87,23 @@ public class BytesUtils {
     return buffer;
   }
 
-  public static byte[][] splitPreserveAllTokens(byte[] str, char separatorChar, int[] target) {
-    return splitWorker(str, 0, -1, separatorChar, true, target);
+  public static byte[][] splitPreserveAllTokens(byte[] str, char separatorChar, int[] target, int numColumns) {
+    return splitWorker(str, 0, -1, separatorChar, target, numColumns);
   }
 
-  public static byte[][] splitPreserveAllTokens(byte[] str, int offset, int length, char separatorChar, int[] target) {
-    return splitWorker(str, offset, length, separatorChar, true, target);
+  public static byte[][] splitPreserveAllTokens(byte[] str, int offset, int length, byte[] separator, int[] target, int numColumns) {
+    return splitWorker(str, offset, length, separator, target, numColumns);
   }
 
-  public static byte[][] splitPreserveAllTokens(byte[] str, char separatorChar) {
-    return splitWorker(str, 0, -1, separatorChar, true, null);
+  public static byte[][] splitPreserveAllTokens(byte[] str, char separatorChar, int numColumns) {
+    return splitWorker(str, 0, -1, separatorChar, null, numColumns);
   }
 
-  public static byte[][] splitPreserveAllTokens(byte[] str, int length, char separatorChar) {
-    return splitWorker(str, 0, length, separatorChar, true, null);
+  private static byte[][] splitWorker(byte[] str, int offset, int length, char separatorChar,
+                                      int[] target, int numColumns) {
+    return splitWorker(str, offset, length, new byte[] {(byte)separatorChar}, target, numColumns);
   }
-
+  
   /**
    * Performs the logic for the <code>split</code> and
    * <code>splitPreserveAllTokens</code> methods that do not return a
@@ -109,75 +111,96 @@ public class BytesUtils {
    *
    * @param str  the String to parse, may be <code>null</code>
    * @param length amount of bytes to str
-   * @param separatorChar the ascii separate character
-   * @param preserveAllTokens if <code>true</code>, adjacent separators are
-   * treated as empty token separators; if <code>false</code>, adjacent
-   * separators are treated as one separator.
+   * @param separator the ascii separate characters
    * @param target the projection target
+   * @param numColumns number of columns to be retrieved              
    * @return an array of parsed Strings, <code>null</code> if null String input
    */
-  private static byte[][] splitWorker(byte[] str, int offset, int length, char separatorChar,
-                                      boolean preserveAllTokens, int[] target) {
-    // Performance tuned for 2.0 (JDK1.4)
-
+  private static byte[][] splitWorker(byte[] str, int offset, int length, byte[] separator, int[] target, int numColumns) {
     if (str == null) {
       return null;
     }
-    int len = length;
-    if (len == 0) {
-      return new byte[1][0];
-    }else if(len < 0){
-      len = str.length - offset;
-    }
-
-    List list = new ArrayList();
-    int i = 0, start = 0;
-    boolean match = false;
-    boolean lastMatch = false;
-    int currentTarget = 0;
-    int currentIndex = 0;
-    while (i < len) {
-      if (str[i + offset] == separatorChar) {
-        if (match || preserveAllTokens) {
-          if (target == null) {
-            byte[] bytes = new byte[i - start];
-            System.arraycopy(str, start + offset, bytes, 0, bytes.length);
-            list.add(bytes);
-          } else if (target.length > currentTarget && currentIndex == target[currentTarget]) {
-            byte[] bytes = new byte[i - start];
-            System.arraycopy(str, start + offset, bytes, 0, bytes.length);
-            list.add(bytes);
-            currentTarget++;
-          } else {
-            list.add(null);
-          }
-          currentIndex++;
-          match = false;
-          lastMatch = true;
-        }
-        start = ++i;
-        continue;
+    if (length == 0) {
+      return new byte[numColumns][0];
+    }
+    if (length < 0) {
+      length = str.length - offset;
+    }
+    int indexMax = 0;
+    if (target != null) {
+      for (int index : target) {
+        indexMax = Math.max(indexMax, index + 1);
       }
-      lastMatch = false;
-      match = true;
-      i++;
-    }
-    if (match || (preserveAllTokens && lastMatch)) {
-      if (target == null) {
-        byte[] bytes = new byte[i - start];
-        System.arraycopy(str, start + offset, bytes, 0, bytes.length);
-        list.add(bytes);
-      } else if (target.length > currentTarget && currentIndex == target[currentTarget]) {
-        byte[] bytes = new byte[i - start];
-        System.arraycopy(str, start + offset, bytes, 0, bytes.length);
-        list.add(bytes); //str.substring(start, i));
-        currentTarget++;
+    } else {
+      indexMax = numColumns;
+    }
+
+    int[][] indices = split(str, offset, length, separator, new int[indexMax][]);
+    byte[][] result = new byte[numColumns][];
+
+    // not-picked -> null, picked but not-exists -> byte[0]
+    if (target != null) {
+      for (int i : target) {
+        int[] index = indices[i];
+        result[i] = index == null ? new byte[0] : Arrays.copyOfRange(str, index[0], index[1]);
+      }
+    } else {
+      for (int i = 0; i < result.length; i++) {
+        int[] index = indices[i];
+        result[i] = index == null ? new byte[0] : Arrays.copyOfRange(str, index[0], index[1]);
+      }
+    }
+    return result;
+  }
+
+  public static int[][] split(byte[] str, int offset, int length, byte[] separator, int[][] indices) {
+    if (indices.length == 0) {
+      return indices;   // trivial
+    }
+    final int limit = offset + length;
+
+    int start = offset;
+    int colIndex = 0;
+    for (int index = offset; index < limit;) {
+      if (onDelimiter(str, index, limit, separator)) {
+        indices[colIndex++] = new int[] {start, index};
+        if (colIndex >= indices.length) {
+          return indices;
+        }
+        index += separator.length;
+        start = index;
       } else {
-        list.add(null);
+        index++;
       }
-      currentIndex++;
     }
-    return (byte[][]) list.toArray(new byte[list.size()][]);
+    if (colIndex < indices.length) {
+      indices[colIndex] = new int[]{start, limit};
+    }
+    return indices;
+  }
+  
+  private static boolean onDelimiter(byte[] input, int offset, int limit, byte[] delimiter) {
+    for (int i = 0; i < delimiter.length; i++) {
+      if (offset + i >= limit || input[offset + i] != delimiter[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+  
+  public static byte[][] splitTrivial(byte[] value, byte delimiter) {
+    List<byte[]> split = new ArrayList<byte[]>();
+    int prev = 0;
+    for (int i = 0; i < value.length; i++) {
+      if (value[i] == delimiter) {
+        split.add(Arrays.copyOfRange(value, prev, i));
+        prev = i + 1;
+      }
+    }
+    if (prev <= value.length) {
+      split.add(Arrays.copyOfRange(value, prev, value.length));
+    }
+    return split.toArray(new byte[split.size()][]);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
index 38c0fd8..d035e4a 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
@@ -186,7 +186,11 @@ public class StringUtils {
   public static String unicodeEscapedDelimiter(String value) {
     try {
       String delimiter = StringEscapeUtils.unescapeJava(value);
-      return unicodeEscapedDelimiter(delimiter.charAt(0));
+      StringBuilder builder = new StringBuilder();
+      for (char achar : delimiter.toCharArray()) {
+        builder.append(unicodeEscapedDelimiter(achar));
+      }
+      return builder.toString();
     } catch (Throwable e) {
     }
     return value;

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-common/src/test/java/org/apache/tajo/util/TestStringUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/util/TestStringUtil.java b/tajo-common/src/test/java/org/apache/tajo/util/TestStringUtil.java
index 5272586..c4329a1 100644
--- a/tajo-common/src/test/java/org/apache/tajo/util/TestStringUtil.java
+++ b/tajo-common/src/test/java/org/apache/tajo/util/TestStringUtil.java
@@ -103,7 +103,7 @@ public class TestStringUtil {
     char separatorChar = '|';
 
     String[] textArray = org.apache.commons.lang.StringUtils.splitPreserveAllTokens(text, separatorChar);
-    byte[][] bytesArray =  BytesUtils.splitPreserveAllTokens(text.getBytes(), separatorChar);
+    byte[][] bytesArray =  BytesUtils.splitPreserveAllTokens(text.getBytes(), separatorChar, 3);
 
     assertEquals(textArray.length, bytesArray.length);
     for (int i = 0; i < textArray.length; i++){
@@ -118,7 +118,7 @@ public class TestStringUtil {
     char separatorChar = '|';
 
     String[] textArray = org.apache.commons.lang.StringUtils.splitPreserveAllTokens(text, separatorChar);
-    byte[][] bytesArray =  BytesUtils.splitPreserveAllTokens(text.getBytes(), separatorChar, target);
+    byte[][] bytesArray =  BytesUtils.splitPreserveAllTokens(text.getBytes(), separatorChar, target, 3);
 
     assertEquals(textArray.length, bytesArray.length);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index 4e4b710..876e3e4 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -238,8 +238,9 @@ public class ExprTestBase {
         targetIdx[i] = i;
       }
 
-      lazyTuple =
-          new LazyTuple(inputSchema, BytesUtils.splitPreserveAllTokens(csvTuple.getBytes(), delimiter, targetIdx),0);
+      byte[][] tokens = BytesUtils.splitPreserveAllTokens(
+          csvTuple.getBytes(), delimiter, targetIdx, inputSchema.size());
+      lazyTuple = new LazyTuple(inputSchema, tokens,0);
       vtuple = new VTuple(inputSchema.size());
       for (int i = 0; i < inputSchema.size(); i++) {
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
index 9ba8a56..dd93dd1 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
@@ -635,4 +635,28 @@ public class TestSelectQuery extends QueryTestCaseBase {
       testingCluster.getConfiguration().setSystemTimezone(TimeZone.getTimeZone("GMT"));
     }
   }
+  
+  @Test
+  public void testMultiBytesDelimiter1() throws Exception {
+    executeDDL("multibytes_delimiter_table1_ddl.sql", "multibytes_delimiter1");
+    try {
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE table1");
+    }
+  }
+  
+  @Test
+  public void testMultiBytesDelimiter2() throws Exception {
+    executeDDL("multibytes_delimiter_table2_ddl.sql", "multibytes_delimiter2");
+    try {
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE table2");
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter1/table1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter1/table1.tbl b/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter1/table1.tbl
new file mode 100644
index 0000000..5acccf6
--- /dev/null
+++ b/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter1/table1.tbl
@@ -0,0 +1,5 @@
+1||ooo||1.1||a
+2||ppp||2.3||
+3||qqq||||
+4||||4.5||
+||xxx||5.6||e
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter2/table2.tbl
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter2/table2.tbl b/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter2/table2.tbl
new file mode 100644
index 0000000..b26cdfd
--- /dev/null
+++ b/tajo-core/src/test/resources/dataset/TestSelectQuery/multibytes_delimiter2/table2.tbl
@@ -0,0 +1,5 @@
+1ㅎoooㅎ1.1ㅎa
+2ㅎpppㅎ2.3ㅎ
+3ㅎqqqㅎㅎ
+4ㅎㅎ4.5ㅎ
+ㅎxxxㅎ5.6ㅎe
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table1_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table1_ddl.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table1_ddl.sql
new file mode 100644
index 0000000..2b4a2ce
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table1_ddl.sql
@@ -0,0 +1,3 @@
+create external table table1 (id int, name text, score float, type text) using csv
+with ('csvfile.delimiter'='||', 'csvfile.null'='NULL') location ${table.path};
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table2_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table2_ddl.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table2_ddl.sql
new file mode 100644
index 0000000..d918ac6
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table2_ddl.sql
@@ -0,0 +1,3 @@
+create external table table2 (id int, name text, score float, type text) using csv
+with ('csvfile.delimiter'='ㅎ', 'csvfile.null'='NULL') location ${table.path};
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter1.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter1.sql
new file mode 100644
index 0000000..bd6b02d
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter1.sql
@@ -0,0 +1 @@
+select * from table1;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter2.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter2.sql
new file mode 100644
index 0000000..66a69ec
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter2.sql
@@ -0,0 +1 @@
+select * from table2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter1.result b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter1.result
new file mode 100644
index 0000000..d8d43b1
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter1.result
@@ -0,0 +1,7 @@
+id,name,score,type
+-------------------------------
+1,ooo,1.1,a
+2,ppp,2.3,
+3,qqq,null,
+4,,4.5,
+null,xxx,5.6,e
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter2.result b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter2.result
new file mode 100644
index 0000000..d8d43b1
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter2.result
@@ -0,0 +1,7 @@
+id,name,score,type
+-------------------------------
+1,ooo,1.1,a
+2,ppp,2.3,
+3,qqq,null,
+4,,4.5,
+null,xxx,5.6,e
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
index c6149f7..fccaf2a 100644
--- a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
@@ -69,7 +69,7 @@ public class TestLazyTuple {
     sb.append(DatumFactory.createInet4("192.168.0.1")).append('|');
     sb.append(new String(nullbytes)).append('|');
     sb.append(NullDatum.get());
-    textRow = BytesUtils.splitPreserveAllTokens(sb.toString().getBytes(), '|');
+    textRow = BytesUtils.splitPreserveAllTokens(sb.toString().getBytes(), '|', 13);
     serde = new TextSerializerDeserializer();
   }
 
@@ -220,7 +220,7 @@ public class TestLazyTuple {
 
   @Test
   public void testInvalidNumber() {
-    byte[][] bytes = BytesUtils.splitPreserveAllTokens(" 1| |2 ||".getBytes(), '|');
+    byte[][] bytes = BytesUtils.splitPreserveAllTokens(" 1| |2 ||".getBytes(), '|', 5);
     Schema schema = new Schema();
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
     schema.addColumn("col2", TajoDataTypes.Type.INT4);

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
index 7ddf09a..c3094fd 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
@@ -85,7 +85,7 @@ public class ColumnMapping {
     for (String eachToken: columnMappingTokens) {
       mappingColumns[index] = new byte[2][];
 
-      byte[][] mappingTokens = BytesUtils.splitPreserveAllTokens(eachToken.trim().getBytes(), ':');
+      byte[][] mappingTokens = BytesUtils.splitTrivial(eachToken.trim().getBytes(), (byte)':');
 
       if (mappingTokens.length == 3) {
         if (mappingTokens[0].length == 0) {
@@ -230,6 +230,10 @@ public class ColumnMapping {
     return numRowKeys;
   }
 
+  public int getNumColumns() {
+    return schema.size();
+  }
+  
   public boolean[] getIsColumnValues() {
     return isColumnValues;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
index 5cae077..ab56252 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
@@ -218,7 +218,8 @@ public class HBaseScanner implements Scanner {
       if (!isBinaryColumns[fieldId] && rowKeyFieldIndexes[fieldId] >= 0) {
         int rowKeyFieldIndex = rowKeyFieldIndexes[fieldId];
 
-        byte[][] rowKeyFields = BytesUtils.splitPreserveAllTokens(value, rowKeyDelimiter);
+        byte[][] rowKeyFields = BytesUtils.splitPreserveAllTokens(
+            value, rowKeyDelimiter, columnMapping.getNumColumns());
 
         if (rowKeyFields.length < rowKeyFieldIndex) {
           return NullDatum.get();

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
index 2a635d8..a9e5bde 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
@@ -1015,7 +1015,8 @@ public class HBaseStorageManager extends StorageManager {
           Tuple endTuple = new VTuple(sortSpecs.length);
           byte[][] rowKeyFields;
           if (sortSpecs.length > 1) {
-            byte[][] splitValues = BytesUtils.splitPreserveAllTokens(eachEndKey, columnMapping.getRowKeyDelimiter());
+            byte[][] splitValues = BytesUtils.splitPreserveAllTokens(
+                eachEndKey, columnMapping.getRowKeyDelimiter(), columnMapping.getNumColumns());
             if (splitValues.length == sortSpecs.length) {
               rowKeyFields = splitValues;
             } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
index dd5366c..bb628b1 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -61,7 +61,7 @@ public class CSVFile {
     private FSDataOutputStream fos;
     private DataOutputStream outputStream;
     private CompressionOutputStream deflateFilter;
-    private char delimiter;
+    private byte[] delimiter;
     private TableStatistics stats = null;
     private Compressor compressor;
     private CompressionCodecFactory codecFactory;
@@ -83,7 +83,7 @@ public class CSVFile {
       this.meta = meta;
       this.schema = schema;
       this.delimiter = StringEscapeUtils.unescapeJava(
-          this.meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
+          this.meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER)).getBytes();
 
       this.columnNum = schema.size();
 
@@ -169,8 +169,8 @@ public class CSVFile {
         rowBytes += serde.serialize(schema.getColumn(i), datum, os, nullChars);
 
         if(columnNum - 1 > i){
-          os.write((byte) delimiter);
-          rowBytes += 1;
+          os.write(delimiter);
+          rowBytes += delimiter.length;
         }
         if (isShuffle) {
           // it is to calculate min/max values, and it is only used for the intermediate file.
@@ -265,7 +265,7 @@ public class CSVFile {
       //Delimiter
       this.delimiter = StringEscapeUtils.unescapeJava(
           meta.getOption(StorageConstants.TEXT_DELIMITER,
-          meta.getOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER))).charAt(0);
+          meta.getOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER))).getBytes();
 
       String nullCharacters = StringEscapeUtils.unescapeJava(
           meta.getOption(StorageConstants.TEXT_NULL,
@@ -279,7 +279,7 @@ public class CSVFile {
     }
 
     private final static int DEFAULT_PAGE_SIZE = 256 * 1024;
-    private char delimiter;
+    private byte[] delimiter;
     private FileSystem fs;
     private FSDataInputStream fis;
     private InputStream is; //decompressd stream
@@ -476,7 +476,7 @@ public class CSVFile {
         }
 
         byte[][] cells = BytesUtils.splitPreserveAllTokens(buffer.getData(), startOffsets.get(currentIdx),
-            rowLengthList.get(currentIdx), delimiter, targetColumnIndexes);
+            rowLengthList.get(currentIdx), delimiter, targetColumnIndexes, schema.size());
         currentIdx++;
         return new LazyTuple(schema, cells, offset, nullChars, serde);
       } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/7f056955/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
index 74563ff..92a041c 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
@@ -171,7 +171,8 @@ public class SequenceFileScanner extends FileScanner {
       } else {
         Text text = new Text();
         reader.getCurrentValue(text);
-        cells = BytesUtils.splitPreserveAllTokens(text.getBytes(), delimiter, projectionMap);
+        cells = BytesUtils.splitPreserveAllTokens(text.getBytes(), 
+            delimiter, projectionMap, schema.getColumns().size());
         totalBytes += (long)text.getBytes().length;
         tuple = new LazyTuple(schema, cells, 0, nullChars, serde);
       }


[07/13] tajo git commit: TAJO-1337: Implements common modules to handle RESTful API

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProviderTest.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProviderTest.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProviderTest.java
new file mode 100644
index 0000000..1511068
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProviderTest.java
@@ -0,0 +1,66 @@
+/**
+ * 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.ws.rs.netty;
+
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelInboundHandler;
+import org.apache.tajo.ws.rs.netty.testapp1.TestApplication1;
+import org.glassfish.jersey.server.ApplicationHandler;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class NettyRestHandlerContainerProviderTest {
+
+  private NettyRestHandlerContainerProvider provider;
+  private ApplicationHandler applicationHandler;
+
+  @Before
+  public void setUp() throws Exception {
+    provider = new NettyRestHandlerContainerProvider();
+    applicationHandler = new ApplicationHandler(new TestApplication1());
+  }
+
+  @Test
+  public void testCreation() throws Exception {
+    ChannelHandler handler = provider.createContainer(ChannelHandler.class, applicationHandler);
+
+    assertNotNull(handler);
+
+    ChannelInboundHandler inboundHandler = provider.createContainer(ChannelInboundHandler.class, applicationHandler);
+
+    assertNotNull(inboundHandler);
+
+    NettyRestHandlerContainer container = provider.createContainer(NettyRestHandlerContainer.class, applicationHandler);
+
+    assertNotNull(container);
+  }
+
+  @Test
+  public void testNullCreation() throws Exception {
+    String stringValue = provider.createContainer(String.class, applicationHandler);
+
+    assertNull(stringValue);
+
+    Object objectValue = provider.createContainer(Object.class, applicationHandler);
+
+    assertNull(objectValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestServerTest.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestServerTest.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestServerTest.java
new file mode 100644
index 0000000..d8a57bf
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/NettyRestServerTest.java
@@ -0,0 +1,137 @@
+/**
+ * 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.ws.rs.netty;
+
+import org.apache.tajo.ws.rs.netty.gson.GsonFeature;
+import org.apache.tajo.ws.rs.netty.testapp1.TestApplication1;
+import org.apache.tajo.ws.rs.netty.testapp1.TestResource1;
+import org.apache.tajo.ws.rs.netty.testapp2.Directory;
+import org.apache.tajo.ws.rs.netty.testapp2.FileManagementApplication;
+import org.glassfish.jersey.server.ResourceConfig;
+import org.junit.Test;
+
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.GenericType;
+import javax.ws.rs.core.MediaType;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.URI;
+import java.util.Collection;
+
+import static org.junit.Assert.*;
+
+public class NettyRestServerTest {
+
+  @Test
+  public void testNettyRestServerCreation() throws Exception {
+    ResourceConfig resourceConfig = ResourceConfig.forApplicationClass(TestApplication1.class);
+    ServerSocket serverSocket = new ServerSocket(0);
+    int availPort = serverSocket.getLocalPort();
+    serverSocket.close();
+    URI baseUri = new URI("http://localhost:"+availPort+"/rest");
+
+    NettyRestServer restServer = NettyRestServerFactory.createNettyRestServer(baseUri, resourceConfig, 3);
+
+    assertNotNull(restServer);
+    assertNotNull(restServer.getHandler());
+    assertNotNull(restServer.getChannel());
+    assertNotNull(restServer.getListenAddress());
+
+    InetSocketAddress listeningAddress = restServer.getListenAddress();
+
+    assertEquals(availPort, listeningAddress.getPort());
+  }
+
+  @Test
+  public void testTextPlainApplication() throws Exception {
+    ResourceConfig resourceConfig = ResourceConfig.forApplicationClass(TestApplication1.class);
+    ServerSocket serverSocket = new ServerSocket(0);
+    int availPort = serverSocket.getLocalPort();
+    serverSocket.close();
+    URI baseUri = new URI("http://localhost:"+availPort+"/rest");
+
+    NettyRestServer restServer = NettyRestServerFactory.createNettyRestServer(baseUri, resourceConfig, 3);
+
+    try {
+      WebTarget webTarget = ClientBuilder.newClient().target(baseUri + "/testapp1");
+
+      assertEquals(TestResource1.outputMessage, webTarget.request(MediaType.TEXT_PLAIN).get(String.class));
+    } finally {
+      restServer.shutdown();
+    }
+  }
+
+  protected Directory createDirectory1() {
+    Directory newDirectory = new Directory();
+
+    newDirectory.setName("newdir1");
+    newDirectory.setOwner("owner1");
+    newDirectory.setGroup("group1");
+
+    return newDirectory;
+  }
+
+  @Test
+  public void testFileMgmtApplication() throws Exception {
+    ResourceConfig resourceConfig = ResourceConfig.forApplicationClass(FileManagementApplication.class)
+        .register(GsonFeature.class);
+    ServerSocket serverSocket = new ServerSocket(0);
+    int availPort = serverSocket.getLocalPort();
+    serverSocket.close();
+    URI baseUri = new URI("http://localhost:"+availPort+"/rest");
+    URI directoriesUri = new URI(baseUri + "/directories");
+    Client restClient = ClientBuilder.newBuilder()
+        .register(GsonFeature.class).build();
+
+    NettyRestServer restServer = NettyRestServerFactory.createNettyRestServer(baseUri, resourceConfig, 3);
+
+    try {
+      Directory directory1 = createDirectory1();
+      Directory savedDirectory = restClient.target(directoriesUri)
+          .request().post(Entity.entity(directory1, MediaType.APPLICATION_JSON_TYPE), Directory.class);
+
+      assertNotNull(savedDirectory);
+      assertNotNull(savedDirectory.getName());
+
+      Directory fetchedDirectory = restClient.target(directoriesUri).path("{name}")
+          .resolveTemplate("name", directory1.getName()).request().get(Directory.class);
+
+      assertEquals(directory1.getName(), fetchedDirectory.getName());
+      assertEquals(directory1.getOwner(), fetchedDirectory.getOwner());
+      assertEquals(directory1.getGroup(), fetchedDirectory.getGroup());
+
+      GenericType<Collection<Directory>> directoryType = new GenericType<Collection<Directory>>(Collection.class);
+      Collection<Directory> directories = restClient.target(directoriesUri).request().get(directoryType);
+
+      assertEquals(1, directories.size());
+
+      restClient.target(directoriesUri).path("{name}").resolveTemplate("name", directory1.getName())
+          .request().delete();
+
+      directories = restClient.target(directoriesUri).request().get(directoryType);
+
+      assertTrue(directories.isEmpty());
+    } finally {
+      restServer.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestApplication1.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestApplication1.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestApplication1.java
new file mode 100644
index 0000000..3531b3a
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestApplication1.java
@@ -0,0 +1,38 @@
+/**
+ * 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.ws.rs.netty.testapp1;
+
+import javax.ws.rs.core.Application;
+import java.util.HashSet;
+import java.util.Set;
+
+public class TestApplication1 extends Application {
+
+  private final Set<Class<?>> classes;
+
+  public TestApplication1() {
+    classes = new HashSet<Class<?>>();
+    classes.add(TestResource1.class);
+  }
+
+  @Override
+  public Set<Class<?>> getClasses() {
+    return classes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestResource1.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestResource1.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestResource1.java
new file mode 100644
index 0000000..302e217
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp1/TestResource1.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ws.rs.netty.testapp1;
+
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+@Path("testapp1")
+public class TestResource1 {
+
+  public static String outputMessage = "TestApplication1";
+
+  @GET
+  @Produces(MediaType.TEXT_PLAIN)
+  public String getApplicationName() {
+    return outputMessage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesDao.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesDao.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesDao.java
new file mode 100644
index 0000000..0e82e00
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesDao.java
@@ -0,0 +1,39 @@
+/**
+ * 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.ws.rs.netty.testapp2;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class DirectoriesDao {
+  private static DirectoriesDao instance = new DirectoriesDao();
+
+  private final Map<String, Directory> directoryMap = new ConcurrentHashMap<String, Directory>();
+
+  private DirectoriesDao() {
+  }
+
+  public static DirectoriesDao getInstance() {
+    return instance;
+  }
+
+  public Map<String, Directory> getDirectoryMap() {
+    return directoryMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesResource.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesResource.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesResource.java
new file mode 100644
index 0000000..40f1ced
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/DirectoriesResource.java
@@ -0,0 +1,85 @@
+/**
+ * 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.ws.rs.netty.testapp2;
+
+import javax.ws.rs.*;
+import javax.ws.rs.core.*;
+import java.net.URI;
+import java.util.Collection;
+
+@Path("/directories")
+public class DirectoriesResource {
+
+  @Context
+  UriInfo uriInfo;
+
+  @GET
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getAllDirectories() {
+    Collection<Directory> directories = DirectoriesDao.getInstance().getDirectoryMap().values();
+    GenericEntity<Collection<Directory>> dirEntities =
+        new GenericEntity<Collection<Directory>>(directories, Collection.class);
+    return Response.ok(dirEntities).build();
+  }
+
+  @POST
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response createDirectory(Directory directory) {
+    String directoryName = directory.getName();
+
+    if (directoryName == null || directoryName.isEmpty()) {
+      return Response.status(Response.Status.BAD_REQUEST).build();
+    }
+
+    DirectoriesDao.getInstance().getDirectoryMap().put(directoryName, directory);
+
+    UriBuilder uriBuilder = uriInfo.getBaseUriBuilder();
+    URI directoryUri = uriBuilder.path(DirectoriesResource.class)
+        .path(DirectoriesResource.class, "getDirectoryByName")
+        .build(directoryName);
+
+    return Response.created(directoryUri).entity(directory).build();
+  }
+
+  @GET
+  @Path("{name}")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getDirectoryByName(@PathParam("name") String directoryName) {
+    Directory directory = DirectoriesDao.getInstance().getDirectoryMap().get(directoryName);
+
+    if (directory == null) {
+      return Response.status(Response.Status.NOT_FOUND).build();
+    }
+
+    return Response.ok(directory).build();
+  }
+
+  @DELETE
+  @Path("{name}")
+  public Response deleteDirectory(@PathParam("name") String directoryName) {
+    if (!DirectoriesDao.getInstance().getDirectoryMap().containsKey(directoryName)) {
+      return Response.status(Response.Status.NOT_FOUND).build();
+    }
+
+    DirectoriesDao.getInstance().getDirectoryMap().remove(directoryName);
+
+    return Response.ok().build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/Directory.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/Directory.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/Directory.java
new file mode 100644
index 0000000..3851020
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/Directory.java
@@ -0,0 +1,52 @@
+/**
+ * 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.ws.rs.netty.testapp2;
+
+import java.io.Serializable;
+
+public class Directory implements Serializable {
+
+  private String name;
+  private String owner;
+  private String group;
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getOwner() {
+    return owner;
+  }
+
+  public void setOwner(String owner) {
+    this.owner = owner;
+  }
+
+  public String getGroup() {
+    return group;
+  }
+
+  public void setGroup(String group) {
+    this.group = group;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/FileManagementApplication.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/FileManagementApplication.java b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/FileManagementApplication.java
new file mode 100644
index 0000000..49026a0
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/test/java/org/apache/tajo/ws/rs/netty/testapp2/FileManagementApplication.java
@@ -0,0 +1,35 @@
+/**
+ * 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.ws.rs.netty.testapp2;
+
+import javax.ws.rs.core.Application;
+import java.util.HashSet;
+import java.util.Set;
+
+public class FileManagementApplication extends Application {
+
+  @Override
+  public Set<Class<?>> getClasses() {
+    Set<Class<?>> classes = new HashSet<Class<?>>();
+
+    classes.add(DirectoriesResource.class);
+
+    return classes;
+  }
+}


[08/13] tajo git commit: TAJO-1337: Implements common modules to handle RESTful API

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ServerCallable.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ServerCallable.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ServerCallable.java
new file mode 100644
index 0000000..fb1cec2
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ServerCallable.java
@@ -0,0 +1,162 @@
+/**
+ * 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.rpc;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.protobuf.ServiceException;
+
+public abstract class ServerCallable<T> {
+  protected InetSocketAddress addr;
+  protected long startTime;
+  protected long endTime;
+  protected Class<?> protocol;
+  protected boolean asyncMode;
+  protected boolean closeConn;
+  protected RpcConnectionPool connPool;
+
+  public abstract T call(NettyClientBase client) throws Exception;
+
+  public ServerCallable(RpcConnectionPool connPool,  InetSocketAddress addr, Class<?> protocol, boolean asyncMode) {
+    this(connPool, addr, protocol, asyncMode, false);
+  }
+
+  public ServerCallable(RpcConnectionPool connPool, InetSocketAddress addr, Class<?> protocol,
+                        boolean asyncMode, boolean closeConn) {
+    this.connPool = connPool;
+    this.addr = addr;
+    this.protocol = protocol;
+    this.asyncMode = asyncMode;
+    this.closeConn = closeConn;
+  }
+
+  public void beforeCall() {
+    this.startTime = System.currentTimeMillis();
+  }
+
+  public long getStartTime(){
+    return startTime;
+  }
+
+  public void afterCall() {
+    this.endTime = System.currentTimeMillis();
+  }
+
+  public long getEndTime(){
+    return endTime;
+  }
+
+  boolean abort = false;
+  public void abort() {
+    abort = true;
+  }
+  /**
+   * Run this instance with retries, timed waits,
+   * and refinds of missing regions.
+   *
+   * @param <T> the type of the return value
+   * @return an object of type T
+   * @throws com.google.protobuf.ServiceException if a remote or network exception occurs
+   */
+  public T withRetries() throws ServiceException {
+    //TODO configurable
+    final long pause = 500; //ms
+    final int numRetries = 3;
+    List<Throwable> exceptions = new ArrayList<Throwable>();
+
+    for (int tries = 0; tries < numRetries; tries++) {
+      NettyClientBase client = null;
+      try {
+        beforeCall();
+        if(addr != null) {
+          client = connPool.getConnection(addr, protocol, asyncMode);
+        }
+        return call(client);
+      } catch (IOException ioe) {
+        exceptions.add(ioe);
+        if(abort) {
+          throw new ServiceException(ioe.getMessage(), ioe);
+        }
+        if (tries == numRetries - 1) {
+          throw new ServiceException("Giving up after tries=" + tries, ioe);
+        }
+      } catch (Throwable t) {
+        throw new ServiceException(t);
+      } finally {
+        afterCall();
+        if(closeConn) {
+          connPool.closeConnection(client);
+        } else {
+          connPool.releaseConnection(client);
+        }
+      }
+      try {
+        Thread.sleep(pause * (tries + 1));
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new ServiceException("Giving up after tries=" + tries, e);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Run this instance against the server once.
+   * @param <T> the type of the return value
+   * @return an object of type T
+   * @throws java.io.IOException if a remote or network exception occurs
+   * @throws RuntimeException other unspecified error
+   */
+  public T withoutRetries() throws IOException, RuntimeException {
+    NettyClientBase client = null;
+    try {
+      beforeCall();
+      client = connPool.getConnection(addr, protocol, asyncMode);
+      return call(client);
+    } catch (Throwable t) {
+      Throwable t2 = translateException(t);
+      if (t2 instanceof IOException) {
+        throw (IOException)t2;
+      } else {
+        throw new RuntimeException(t2);
+      }
+    } finally {
+      afterCall();
+      if(closeConn) {
+        connPool.closeConnection(client);
+      } else {
+        connPool.releaseConnection(client);
+      }
+    }
+  }
+
+  private static Throwable translateException(Throwable t) throws IOException {
+    if (t instanceof UndeclaredThrowableException) {
+      t = t.getCause();
+    }
+    if (t instanceof RemoteException && t.getCause() != null) {
+      t = t.getCause();
+    }
+    return t;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/TajoServiceException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/TajoServiceException.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/TajoServiceException.java
new file mode 100644
index 0000000..113d181
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/TajoServiceException.java
@@ -0,0 +1,58 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang.exception.ExceptionUtils;
+
+public class TajoServiceException extends ServiceException {
+  private String traceMessage;
+  private String protocol;
+  private String remoteAddress;
+
+  public TajoServiceException(String message) {
+    super(message);
+  }
+  public TajoServiceException(String message, String traceMessage) {
+    super(message);
+    this.traceMessage = traceMessage;
+  }
+
+  public TajoServiceException(String message, Throwable cause, String protocol, String remoteAddress) {
+    super(message, cause);
+
+    this.protocol = protocol;
+    this.remoteAddress = remoteAddress;
+  }
+
+  public String getTraceMessage() {
+    if(traceMessage == null && getCause() != null){
+      this.traceMessage = ExceptionUtils.getStackTrace(getCause());
+    }
+    return traceMessage;
+  }
+
+  public String getProtocol() {
+    return protocol;
+  }
+
+  public String getRemoteAddress() {
+    return remoteAddress;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/proto/DummyProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/proto/DummyProtos.proto b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/DummyProtos.proto
new file mode 100644
index 0000000..f53f0d6
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/DummyProtos.proto
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2012 Database Lab., Korea Univ.
+ *
+ * Licensed 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.
+ */
+
+option java_package = "org.apache.tajo.rpc.test";
+option java_outer_classname = "DummyProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+message MulRequest1 {
+	required int32 x1 = 1;
+	required int32 x2 = 2;
+}
+
+message MulRequest2 {
+	required int32 x1 = 1;
+	required int32 x2 = 2;
+}
+
+message MulResponse {
+	required int32 result1 = 1;
+	required int32 result2 = 2;
+}
+
+message InnerNode {
+	required string instr = 1;
+}
+
+message InnerRequest {
+	repeated InnerNode nodes = 1;
+}
+
+message InnerResponse {
+	repeated InnerNode nodes = 1;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/proto/RpcProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/proto/RpcProtos.proto b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/RpcProtos.proto
new file mode 100644
index 0000000..69f43ed
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/RpcProtos.proto
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2012 Database Lab., Korea Univ.
+ *
+ * Licensed 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.
+ */
+
+option java_package = "org.apache.tajo.rpc";
+option java_outer_classname = "RpcProtos";
+
+message RpcRequest {
+  required int32 id = 1;
+  required string method_name = 2;
+  optional bytes request_message = 3;
+}
+
+message RpcResponse {
+  required int32 id = 1;
+  optional bytes response_message = 2;
+  optional string error_class = 3;
+  optional string error_message = 4;
+  optional string error_trace = 5;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtocol.proto b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtocol.proto
new file mode 100644
index 0000000..58640ea
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtocol.proto
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2012 Database Lab., Korea Univ.
+ *
+ * Licensed 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.
+ */
+
+option java_package = "org.apache.tajo.rpc.test";
+option java_outer_classname = "DummyProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "TestProtos.proto";
+
+service DummyProtocolService {
+  rpc sum (SumRequest) returns (SumResponse);
+  rpc echo (EchoMessage) returns (EchoMessage);
+  rpc getError (EchoMessage) returns (EchoMessage);
+  rpc getNull (EchoMessage) returns (EchoMessage);
+  rpc deley (EchoMessage) returns (EchoMessage);
+  rpc throwException (EchoMessage) returns (EchoMessage);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtos.proto b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtos.proto
new file mode 100644
index 0000000..5001c0e
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/proto/TestProtos.proto
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2012 Database Lab., Korea Univ.
+ *
+ * Licensed 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.
+ */
+
+option java_package = "org.apache.tajo.rpc.test";
+option java_outer_classname = "TestProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+message EchoMessage {
+  required string message = 1;
+}
+
+message SumRequest {
+  required int32 x1 = 1;
+  required int64 x2 = 2;
+  required double x3 = 3;
+  required float x4 = 4;
+}
+
+message SumResponse {
+  required double result = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/test/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/test/java/log4j.properties b/tajo-rpc/tajo-rpc-protobuf/src/test/java/log4j.properties
new file mode 100644
index 0000000..2c4d991
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/test/java/log4j.properties
@@ -0,0 +1,25 @@
+##
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %p: %c (%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
new file mode 100644
index 0000000..a974a65
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
@@ -0,0 +1,345 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.test.DummyProtocol;
+import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.Interface;
+import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
+import org.apache.tajo.rpc.test.TestProtos.SumRequest;
+import org.apache.tajo.rpc.test.TestProtos.SumResponse;
+import org.apache.tajo.rpc.test.impl.DummyProtocolAsyncImpl;
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExternalResource;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.*;
+
+public class TestAsyncRpc {
+  private static Log LOG = LogFactory.getLog(TestAsyncRpc.class);
+  private static String MESSAGE = "TestAsyncRpc";
+
+  double sum;
+  String echo;
+
+  AsyncRpcServer server;
+  AsyncRpcClient client;
+  Interface stub;
+  DummyProtocolAsyncImpl service;
+  int retries;
+  
+  @Retention(RetentionPolicy.RUNTIME)
+  @Target(ElementType.METHOD)
+  @interface SetupRpcConnection {
+    boolean setupRpcServer() default true;
+    boolean setupRpcClient() default true;
+  }
+  
+  @Rule
+  public ExternalResource resource = new ExternalResource() {
+    
+    private Description description;
+
+    @Override
+    public Statement apply(Statement base, Description description) {
+      this.description = description;
+      return super.apply(base, description);
+    }
+
+    @Override
+    protected void before() throws Throwable {
+      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
+
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
+        setUpRpcServer();
+      }
+      
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
+        setUpRpcClient();
+      }
+    }
+
+    @Override
+    protected void after() {
+      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
+
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
+        try {
+          tearDownRpcClient();
+        } catch (Exception e) {
+          fail(e.getMessage());
+        }
+      }
+      
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
+        try {
+          tearDownRpcServer();
+        } catch (Exception e) {
+          fail(e.getMessage());
+        }
+      }
+    }
+    
+  };
+  
+  public void setUpRpcServer() throws Exception {
+    service = new DummyProtocolAsyncImpl();
+    server = new AsyncRpcServer(DummyProtocol.class,
+        service, new InetSocketAddress("127.0.0.1", 0), 2);
+    server.start();
+  }
+  
+  public void setUpRpcClient() throws Exception {
+    retries = 1;
+
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.getConnectAddress(server.getListenAddress()),
+              DummyProtocol.class, true);
+    client = new AsyncRpcClient(rpcConnectionKey, retries);
+    client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT);
+    stub = client.getStub();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    RpcChannelFactory.shutdownGracefully();
+  }
+  
+  public void tearDownRpcServer() throws Exception {
+    if(server != null) {
+      server.shutdown();
+      server = null;
+    }
+  }
+  
+  public void tearDownRpcClient() throws Exception {
+    if(client != null) {
+      client.close();
+      client = null;
+    }
+  }
+
+  boolean calledMarker = false;
+
+  @Test
+  public void testRpc() throws Exception {
+
+    SumRequest sumRequest = SumRequest.newBuilder()
+        .setX1(1)
+        .setX2(2)
+        .setX3(3.15d)
+        .setX4(2.0f).build();
+
+    stub.sum(null, sumRequest, new RpcCallback<SumResponse>() {
+      @Override
+      public void run(SumResponse parameter) {
+        sum = parameter.getResult();
+        assertTrue(8.15d == sum);
+      }
+    });
+
+
+    EchoMessage echoMessage = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    RpcCallback<EchoMessage> callback = new RpcCallback<EchoMessage>() {
+      @Override
+      public void run(EchoMessage parameter) {
+        echo = parameter.getMessage();
+        assertEquals(MESSAGE, echo);
+        calledMarker = true;
+      }
+    };
+    stub.echo(null, echoMessage, callback);
+    Thread.sleep(1000);
+    assertTrue(calledMarker);
+  }
+
+  private CountDownLatch testNullLatch;
+
+  @Test
+  public void testGetNull() throws Exception {
+    testNullLatch = new CountDownLatch(1);
+    stub.getNull(null, null, new RpcCallback<EchoMessage>() {
+      @Override
+      public void run(EchoMessage parameter) {
+        assertNull(parameter);
+        LOG.info("testGetNull retrieved");
+        testNullLatch.countDown();
+      }
+    });
+    assertTrue(testNullLatch.await(1000, TimeUnit.MILLISECONDS));
+    assertTrue(service.getNullCalled);
+  }
+
+  @Test
+  public void testCallFuture() throws Exception {
+    EchoMessage echoMessage = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
+    stub.deley(null, echoMessage, future);
+
+    assertFalse(future.isDone());
+    assertEquals(future.get(), echoMessage);
+    assertTrue(future.isDone());
+  }
+
+  @Test
+  public void testCallFutureTimeout() throws Exception {
+    boolean timeout = false;
+    try {
+      EchoMessage echoMessage = EchoMessage.newBuilder()
+          .setMessage(MESSAGE).build();
+      CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
+      stub.deley(null, echoMessage, future);
+
+      assertFalse(future.isDone());
+      future.get(1, TimeUnit.SECONDS);
+    } catch (TimeoutException te) {
+      timeout = true;
+    }
+    assertTrue(timeout);
+  }
+
+  @Test
+  public void testCallFutureDisconnected() throws Exception {
+    EchoMessage echoMessage = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
+
+    tearDownRpcServer();
+
+    stub.echo(future.getController(), echoMessage, future);
+    EchoMessage response = future.get();
+
+    assertNull(response);
+    assertTrue(future.getController().failed());
+    assertTrue(future.getController().errorText() != null);
+  }
+
+  @Test
+  public void testStubDisconnected() throws Exception {
+
+    EchoMessage echoMessage = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
+
+    if (server != null) {
+      server.shutdown(true);
+      server = null;
+    }
+
+    stub = client.getStub();
+    stub.echo(future.getController(), echoMessage, future);
+    EchoMessage response = future.get();
+
+    assertNull(response);
+    assertTrue(future.getController().failed());
+    assertTrue(future.getController().errorText() != null);
+  }
+
+  @Test
+  @SetupRpcConnection(setupRpcServer=false,setupRpcClient=false)
+  public void testConnectionRetry() throws Exception {
+    retries = 10;
+    ServerSocket serverSocket = new ServerSocket(0);
+    final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort());
+    serverSocket.close();
+    service = new DummyProtocolAsyncImpl();
+
+    EchoMessage echoMessage = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
+
+    //lazy startup
+    Thread serverThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          Thread.sleep(1000);
+          server = new AsyncRpcServer(DummyProtocol.class,
+              service, address, 2);
+        } catch (Exception e) {
+          fail(e.getMessage());
+        }
+        server.start();
+      }
+    });
+    serverThread.start();
+
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, true);
+    client = new AsyncRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
+    stub = client.getStub();
+    stub.echo(future.getController(), echoMessage, future);
+
+    assertFalse(future.isDone());
+    assertEquals(echoMessage, future.get());
+    assertTrue(future.isDone());
+  }
+
+  @Test
+  public void testConnectionFailure() throws Exception {
+    InetSocketAddress address = new InetSocketAddress("test", 0);
+    try {
+      RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, true);
+      NettyClientBase client = new AsyncRpcClient(rpcConnectionKey, retries);
+      assertFalse(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
+    } catch (Throwable throwable) {
+      fail();
+    }
+  }
+
+  @Test
+  @SetupRpcConnection(setupRpcClient=false)
+  public void testUnresolvedAddress() throws Exception {
+    String hostAndPort = RpcUtils.normalizeInetSocketAddress(server.getListenAddress());
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.createUnresolved(hostAndPort), DummyProtocol.class, true);
+    client = new AsyncRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
+    Interface stub = client.getStub();
+    EchoMessage echoMessage = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
+    stub.deley(null, echoMessage, future);
+
+    assertFalse(future.isDone());
+    assertEquals(future.get(), echoMessage);
+    assertTrue(future.isDone());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
new file mode 100644
index 0000000..10dd766
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
@@ -0,0 +1,349 @@
+/**
+ * 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.rpc;
+
+import org.apache.tajo.rpc.test.DummyProtocol;
+import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.BlockingInterface;
+import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
+import org.apache.tajo.rpc.test.TestProtos.SumRequest;
+import org.apache.tajo.rpc.test.TestProtos.SumResponse;
+import org.apache.tajo.rpc.test.impl.DummyProtocolBlockingImpl;
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExternalResource;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+public class TestBlockingRpc {
+  public static final String MESSAGE = "TestBlockingRpc";
+
+  private BlockingRpcServer server;
+  private BlockingRpcClient client;
+  private BlockingInterface stub;
+  private DummyProtocolBlockingImpl service;
+  private int retries;
+  
+  @Retention(RetentionPolicy.RUNTIME)
+  @Target(ElementType.METHOD)
+  @interface SetupRpcConnection {
+    boolean setupRpcServer() default true;
+    boolean setupRpcClient() default true;
+  }
+  
+  @Rule
+  public ExternalResource resource = new ExternalResource() {
+    
+    private Description description;
+
+    @Override
+    public Statement apply(Statement base, Description description) {
+      this.description = description;
+      return super.apply(base, description);
+    }
+
+    @Override
+    protected void before() throws Throwable {
+      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
+      
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
+        setUpRpcServer();
+      }
+      
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
+        setUpRpcClient();
+      }
+    }
+
+    @Override
+    protected void after() {
+      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
+      
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
+        try {
+          tearDownRpcClient();
+        } catch (Exception e) {
+          fail(e.getMessage());
+        }
+      }
+      
+      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
+        try {
+          tearDownRpcServer();
+        } catch (Exception e) {
+          fail(e.getMessage());
+        }
+      }
+    }
+    
+  };
+  
+  public void setUpRpcServer() throws Exception {
+    service = new DummyProtocolBlockingImpl();
+    server = new BlockingRpcServer(DummyProtocol.class, service,
+        new InetSocketAddress("127.0.0.1", 0), 2);
+    server.start();
+  }
+  
+  public void setUpRpcClient() throws Exception {
+    retries = 1;
+
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.getConnectAddress(server.getListenAddress()),
+              DummyProtocol.class, false);
+    client = new BlockingRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
+    stub = client.getStub();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    RpcChannelFactory.shutdownGracefully();
+  }
+  
+  public void tearDownRpcServer() throws Exception {
+    if(server != null) {
+      server.shutdown();
+      server = null;
+    }
+  }
+  
+  public void tearDownRpcClient() throws Exception {
+    if(client != null) {
+      client.close();
+      client = null;
+    }
+  }
+
+  @Test
+  public void testRpc() throws Exception {
+    SumRequest request = SumRequest.newBuilder()
+        .setX1(1)
+        .setX2(2)
+        .setX3(3.15d)
+        .setX4(2.0f).build();
+    SumResponse response1 = stub.sum(null, request);
+    assertEquals(8.15d, response1.getResult(), 1e-15);
+
+    EchoMessage message = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    EchoMessage response2 = stub.echo(null, message);
+    assertEquals(MESSAGE, response2.getMessage());
+  }
+
+  @Test
+  @SetupRpcConnection(setupRpcClient=false)
+  public void testRpcWithServiceCallable() throws Exception {
+    RpcConnectionPool pool = RpcConnectionPool.getPool();
+    final SumRequest request = SumRequest.newBuilder()
+        .setX1(1)
+        .setX2(2)
+        .setX3(3.15d)
+        .setX4(2.0f).build();
+
+    SumResponse response =
+    new ServerCallable<SumResponse>(pool,
+        server.getListenAddress(), DummyProtocol.class, false) {
+      @Override
+      public SumResponse call(NettyClientBase client) throws Exception {
+        BlockingInterface stub2 = client.getStub();
+        SumResponse response1 = stub2.sum(null, request);
+        return response1;
+      }
+    }.withRetries();
+
+    assertEquals(8.15d, response.getResult(), 1e-15);
+
+    response =
+        new ServerCallable<SumResponse>(pool,
+            server.getListenAddress(), DummyProtocol.class, false) {
+          @Override
+          public SumResponse call(NettyClientBase client) throws Exception {
+            BlockingInterface stub2 = client.getStub();
+            SumResponse response1 = stub2.sum(null, request);
+            return response1;
+          }
+        }.withoutRetries();
+
+    assertTrue(8.15d == response.getResult());
+    pool.close();
+  }
+
+  @Test
+  public void testThrowException() throws Exception {
+    EchoMessage message = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+
+    try {
+      stub.throwException(null, message);
+      fail("RpcCall should throw exception");
+    } catch (Throwable t) {
+      assertTrue(t instanceof TajoServiceException);
+      assertEquals("Exception Test", t.getMessage());
+      TajoServiceException te = (TajoServiceException)t;
+      assertEquals("org.apache.tajo.rpc.test.DummyProtocol", te.getProtocol());
+      assertEquals(server.getListenAddress().getAddress().getHostAddress() + ":" + server.getListenAddress().getPort(),
+          te.getRemoteAddress());
+    }
+  }
+
+  @Test
+  @SetupRpcConnection(setupRpcServer=false,setupRpcClient=false)
+  public void testConnectionRetry() throws Exception {
+    retries = 10;
+    ServerSocket serverSocket = new ServerSocket(0);
+    final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort());
+    serverSocket.close();
+
+    EchoMessage message = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+
+    //lazy startup
+    Thread serverThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          Thread.sleep(1000);
+          server = new BlockingRpcServer(DummyProtocol.class, new DummyProtocolBlockingImpl(), address, 2);
+        } catch (Exception e) {
+          fail(e.getMessage());
+        }
+        server.start();
+      }
+    });
+    serverThread.start();
+
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, false);
+    client = new BlockingRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
+    stub = client.getStub();
+
+    EchoMessage response = stub.echo(null, message);
+    assertEquals(MESSAGE, response.getMessage());
+  }
+
+  @Test
+  public void testConnectionFailed() throws Exception {
+    NettyClientBase client = null;
+    
+    try {
+      int port = server.getListenAddress().getPort() + 1;
+      RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.getConnectAddress(new InetSocketAddress("127.0.0.1", port)),
+              DummyProtocol.class, false);
+      client = new BlockingRpcClient(rpcConnectionKey, retries);
+      assertFalse(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
+      client.close();
+    } catch (Throwable ce){
+      if (client != null) {
+        client.close();
+      }
+      fail();
+    }
+  }
+
+  @Test
+  public void testGetNull() throws Exception {
+    assertNull(stub.getNull(null, null));
+    assertTrue(service.getNullCalled);
+  }
+
+  @Test
+  public void testShutdown() throws Exception {
+    final StringBuilder error = new StringBuilder();
+    Thread callThread = new Thread() {
+      public void run() {
+        try {
+          EchoMessage message = EchoMessage.newBuilder()
+              .setMessage(MESSAGE)
+              .build();
+          stub.deley(null, message);
+        } catch (Exception e) {
+          error.append(e.getMessage());
+        }
+        synchronized(error) {
+          error.notifyAll();
+        }
+      }
+    };
+
+    callThread.start();
+
+    final CountDownLatch latch = new CountDownLatch(1);
+    Thread shutdownThread = new Thread() {
+      public void run() {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+        }
+        try {
+          server.shutdown();
+          server = null;
+          latch.countDown();
+        } catch (Throwable e) {
+          e.printStackTrace();
+        }
+      }
+    };
+    shutdownThread.start();
+
+    assertTrue(latch.await(5 * 1000, TimeUnit.MILLISECONDS));
+
+    assertTrue(latch.getCount() == 0);
+
+    synchronized(error) {
+      error.wait(5 * 1000);
+    }
+
+    if(!error.toString().isEmpty()) {
+      fail(error.toString());
+    }
+  }
+
+  @Test
+  @SetupRpcConnection(setupRpcClient=false)
+  public void testUnresolvedAddress() throws Exception {
+    String hostAndPort = RpcUtils.normalizeInetSocketAddress(server.getListenAddress());
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.createUnresolved(hostAndPort), DummyProtocol.class, false);
+    client = new BlockingRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
+    BlockingInterface stub = client.getStub();
+
+    EchoMessage message = EchoMessage.newBuilder()
+        .setMessage(MESSAGE).build();
+    EchoMessage response2 = stub.echo(null, message);
+    assertEquals(MESSAGE, response2.getMessage());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java
new file mode 100644
index 0000000..0ca7563
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java
@@ -0,0 +1,86 @@
+/**
+ * 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.rpc.test.impl;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.Interface;
+import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
+import org.apache.tajo.rpc.test.TestProtos.SumRequest;
+import org.apache.tajo.rpc.test.TestProtos.SumResponse;
+
+public class DummyProtocolAsyncImpl implements Interface {
+  private static final Log LOG =
+      LogFactory.getLog(DummyProtocolAsyncImpl.class);
+  public boolean getNullCalled = false;
+  public boolean getErrorCalled = false;
+
+  @Override
+  public void sum(RpcController controller, SumRequest request,
+                  RpcCallback<SumResponse> done) {
+
+    SumResponse response = SumResponse.newBuilder().setResult(
+        request.getX1()+request.getX2()+request.getX3()+request.getX4()
+    ).build();
+    done.run(response);
+  }
+
+  @Override
+  public void echo(RpcController controller, EchoMessage request,
+                   RpcCallback<EchoMessage> done) {
+
+    done.run(request);
+  }
+
+  @Override
+  public void getError(RpcController controller, EchoMessage request,
+                       RpcCallback<EchoMessage> done) {
+    LOG.info("noCallback is called");
+    getErrorCalled = true;
+    controller.setFailed(request.getMessage());
+    done.run(request);
+  }
+
+  @Override
+  public void getNull(RpcController controller, EchoMessage request,
+                      RpcCallback<EchoMessage> done) {
+    getNullCalled = true;
+    LOG.info("noCallback is called");
+    done.run(null);
+  }
+
+  @Override
+  public void deley(RpcController controller, EchoMessage request,
+                    RpcCallback<EchoMessage> done) {
+    try {
+      Thread.sleep(3000);
+    } catch (InterruptedException e) {
+      LOG.error(e.getMessage());
+    }
+
+    done.run(request);
+  }
+
+  public void throwException(RpcController controller, EchoMessage request,
+                             RpcCallback<EchoMessage> done) {
+    done.run(request);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
new file mode 100644
index 0000000..8d4b597
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
@@ -0,0 +1,83 @@
+/**
+ * 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.rpc.test.impl;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.BlockingInterface;
+import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
+import org.apache.tajo.rpc.test.TestProtos.SumRequest;
+import org.apache.tajo.rpc.test.TestProtos.SumResponse;
+
+public class DummyProtocolBlockingImpl implements BlockingInterface {
+  private static final Log LOG =
+      LogFactory.getLog(DummyProtocolBlockingImpl.class);
+  public boolean getNullCalled = false;
+  public boolean getErrorCalled = false;
+
+  @Override
+  public SumResponse sum(RpcController controller, SumRequest request)
+      throws ServiceException {
+    return SumResponse.newBuilder().setResult(
+        request.getX1()+request.getX2()+request.getX3()+request.getX4()
+    ).build();
+  }
+
+  @Override
+  public EchoMessage echo(RpcController controller, EchoMessage request)
+      throws ServiceException {
+    return EchoMessage.newBuilder().
+        setMessage(request.getMessage()).build();
+  }
+
+  @Override
+  public EchoMessage getError(RpcController controller, EchoMessage request)
+      throws ServiceException {
+    getErrorCalled = true;
+    controller.setFailed(request.getMessage());
+    return request;
+  }
+
+  @Override
+  public EchoMessage getNull(RpcController controller, EchoMessage request)
+      throws ServiceException {
+    getNullCalled = true;
+    LOG.info("noCallback is called");
+    return null;
+  }
+
+  @Override
+  public EchoMessage deley(RpcController controller, EchoMessage request)
+      throws ServiceException {
+    try {
+      Thread.sleep(3000);
+    } catch (InterruptedException e) {
+      //throw new ServiceException(e.getMessage(), e);
+    }
+
+    return request;
+  }
+
+  public EchoMessage throwException(RpcController controller, EchoMessage request)
+      throws ServiceException {
+    throw new ServiceException("Exception Test");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/pom.xml b/tajo-rpc/tajo-ws-rs/pom.xml
new file mode 100644
index 0000000..a87a67a
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/pom.xml
@@ -0,0 +1,218 @@
+<!--
+  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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <version>0.11.0-SNAPSHOT</version>
+    <groupId>org.apache.tajo</groupId>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <packaging>jar</packaging>
+  <artifactId>tajo-ws-rs</artifactId>
+  <name>Tajo RESTful Container</name>
+  <description>RESTful Container Implementation based on Netty</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <configuration>
+        </configuration>
+        <executions>
+          <execution>
+            <id>create-jar</id>
+            <phase>prepare-package</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-transport</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-codec-http</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-handler</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-rpc-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>javax.ws.rs</groupId>
+      <artifactId>javax.ws.rs-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+  </properties>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                    <echo file="${project.build.directory}/dist-layout-stitching.sh">
+                      run() {
+                      echo "\$ ${@}"
+                      "${@}"
+                      res=$?
+                      if [ $res != 0 ]; then
+                      echo
+                      echo "Failed!"
+                      echo
+                      exit $res
+                      fi
+                      }
+
+                      ROOT=`cd ${basedir}/..;pwd`
+                      echo
+                      echo "Current directory `pwd`"
+                      echo
+                      run rm -rf ${project.artifactId}-${project.version}
+                      run mkdir ${project.artifactId}-${project.version}
+                      run cd ${project.artifactId}-${project.version}
+                      run cp -r ${basedir}/target/${project.artifactId}-${project.version}*.jar .
+                      echo
+                      echo "Tajo RESTful Container dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo
+                    </echo>
+                    <exec executable="sh" dir="${project.build.directory}" failonerror="true">
+                      <arg line="./dist-layout-stitching.sh" />
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+
+</project>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestChannelInitializer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestChannelInitializer.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestChannelInitializer.java
new file mode 100644
index 0000000..a1ea72b
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestChannelInitializer.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.ws.rs.netty;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.handler.codec.http.HttpObjectAggregator;
+import io.netty.handler.codec.http.HttpServerCodec;
+import io.netty.handler.stream.ChunkedWriteHandler;
+
+/**
+ * Default Channel Initializer for Netty Rest server.
+ */
+public class NettyRestChannelInitializer extends ChannelInitializer<Channel> {
+  
+  private ChannelHandler handler;
+  
+  public NettyRestChannelInitializer(ChannelHandler handler) {
+    this.handler = handler;
+  }
+
+  @Override
+  protected void initChannel(Channel channel) throws Exception {
+    ChannelPipeline pipeline = channel.pipeline();
+    
+    pipeline.addLast(new HttpServerCodec());
+    pipeline.addLast(new HttpObjectAggregator(1 << 16));
+    pipeline.addLast(new ChunkedWriteHandler());
+    pipeline.addLast(handler);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainer.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainer.java
new file mode 100644
index 0000000..81d1eeb
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainer.java
@@ -0,0 +1,319 @@
+/**
+ * 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.ws.rs.netty;
+
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.Principal;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import io.netty.buffer.*;
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandler.Sharable;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPromise;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
+import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.util.CharsetUtil;
+import io.netty.util.ReferenceCountUtil;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import javax.ws.rs.core.Application;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.SecurityContext;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.glassfish.hk2.api.ServiceLocator;
+import org.glassfish.jersey.internal.MapPropertiesDelegate;
+import org.glassfish.jersey.server.ApplicationHandler;
+import org.glassfish.jersey.server.ContainerException;
+import org.glassfish.jersey.server.ContainerRequest;
+import org.glassfish.jersey.server.ContainerResponse;
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.server.internal.ConfigHelper;
+import org.glassfish.jersey.server.spi.Container;
+import org.glassfish.jersey.server.spi.ContainerLifecycleListener;
+import org.glassfish.jersey.server.spi.ContainerResponseWriter;
+
+/**
+ * Jersy Container implementation on Netty
+ */
+@Sharable
+public class NettyRestHandlerContainer extends ChannelDuplexHandler implements Container {
+
+  private static Log LOG = LogFactory.getLog(NettyRestHandlerContainer.class);
+
+  private String rootPath;
+
+  private ApplicationHandler applicationHandler;
+  private ContainerLifecycleListener lifecycleListener;
+
+  NettyRestHandlerContainer(Application application) {
+    this(new ApplicationHandler(application));
+  }
+
+  NettyRestHandlerContainer(Application application, ServiceLocator parentLocator) {
+    this(new ApplicationHandler(application, null, parentLocator));
+  }
+
+  NettyRestHandlerContainer(ApplicationHandler appHandler) {
+    applicationHandler = appHandler;
+    lifecycleListener = ConfigHelper.getContainerLifecycleListener(applicationHandler);
+  }
+
+  @Override
+  public ResourceConfig getConfiguration() {
+    return applicationHandler.getConfiguration();
+  }
+
+  @Override
+  public void reload() {
+    reload(getConfiguration());
+  }
+
+  @Override
+  public void reload(ResourceConfig configuration) {
+    lifecycleListener.onShutdown(this);
+    applicationHandler = new ApplicationHandler(configuration);
+    lifecycleListener = ConfigHelper.getContainerLifecycleListener(applicationHandler);
+    lifecycleListener.onReload(this);
+    lifecycleListener.onStartup(this);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NettyRestHandlerContainer reloaded.");
+    }
+  }
+
+  public void setRootPath(String rootPath) {
+    String tempRootPath = rootPath;
+    if (tempRootPath == null || tempRootPath.isEmpty()) {
+      tempRootPath = "/";
+    } else if (tempRootPath.charAt(tempRootPath.length() - 1) != '/') {
+      tempRootPath += "/";
+    }
+    this.rootPath = tempRootPath;
+  }
+  
+  private URI getBaseUri(ChannelHandlerContext ctx, FullHttpRequest request) {
+    URI baseUri;
+    String scheme;
+    
+    if (ctx.pipeline().get(SslHandler.class) == null) {
+      scheme = "http";
+    } else {
+      scheme = "https";
+    }
+    
+    List<String> hosts = request.headers().getAll(HttpHeaders.Names.HOST);
+    try {
+      if (hosts != null && hosts.size() > 0) {
+        baseUri = new URI(scheme + "://" + hosts.get(0) + rootPath);
+      } else {
+        InetSocketAddress localAddress = (InetSocketAddress) ctx.channel().localAddress();
+        baseUri = new URI(scheme, null, localAddress.getHostName(), localAddress.getPort(),
+                    rootPath, null, null);
+      }
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+    
+    return baseUri;
+  }
+
+  protected void messageReceived(ChannelHandlerContext ctx, FullHttpRequest request) throws Exception {
+    URI baseUri = getBaseUri(ctx, request);
+    URI requestUri = baseUri.resolve(request.getUri());
+    ByteBuf responseContent = PooledByteBufAllocator.DEFAULT.buffer();
+    FullHttpResponse response = 
+        new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, responseContent);
+    
+    NettyRestResponseWriter responseWriter = new NettyRestResponseWriter(ctx, response);
+    ContainerRequest containerRequest = new ContainerRequest(baseUri, requestUri, 
+        request.getMethod().name(), getSecurityContext(), new MapPropertiesDelegate());
+    containerRequest.setEntityStream(new ByteBufInputStream(request.content()));
+    
+    HttpHeaders httpHeaders = request.headers();
+    for (String headerName: httpHeaders.names()) {
+      List<String> headerValues = httpHeaders.getAll(headerName);
+      containerRequest.headers(headerName, headerValues);
+    }
+    containerRequest.setWriter(responseWriter);
+    try {
+      applicationHandler.handle(containerRequest);
+    } finally {
+      responseWriter.releaseConnection();
+    }
+  }
+
+  @Override
+  public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+    boolean needRelease = true;
+    try {
+      if (msg instanceof FullHttpRequest) {
+        FullHttpRequest request = (FullHttpRequest) msg;
+        messageReceived(ctx, request);
+      } else {
+        needRelease = false;
+        ctx.fireChannelRead(msg);
+      }
+    } finally {
+      if (needRelease) {
+        ReferenceCountUtil.release(msg);
+      }
+    }
+  }
+
+  private SecurityContext getSecurityContext() {
+    return new SecurityContext() {
+
+      @Override
+      public boolean isUserInRole(String role) {
+        return false;
+      }
+
+      @Override
+      public boolean isSecure() {
+        return false;
+      }
+
+      @Override
+      public Principal getUserPrincipal() {
+        return null;
+      }
+
+      @Override
+      public String getAuthenticationScheme() {
+        return null;
+      }
+    };
+  }
+
+  /**
+   * Internal class for writing content on REST service.
+   */
+  static class NettyRestResponseWriter implements ContainerResponseWriter {
+
+    private final ChannelHandlerContext ctx;
+    private final FullHttpResponse response;
+    private final AtomicBoolean closed;
+
+    public NettyRestResponseWriter(ChannelHandlerContext ctx, FullHttpResponse response) {
+      this.ctx = ctx;
+      this.response = response;
+      this.closed = new AtomicBoolean(false);
+    }
+
+    @Override
+    public void commit() {
+      if (closed.compareAndSet(false, true)) {
+        ctx.write(response);
+        sendLastHttpContent();
+      }
+    }
+
+    @Override
+    public boolean enableResponseBuffering() {
+      return false;
+    }
+
+    @Override
+    public void failure(Throwable error) {
+      try {
+        sendError(HttpResponseStatus.INTERNAL_SERVER_ERROR, error);
+      } finally {
+        if (ctx.channel().isActive()) {
+          ctx.close();
+        }
+      }
+    }
+    
+    private void sendError(HttpResponseStatus status, final Throwable error) {
+      FullHttpResponse response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, status,
+          Unpooled.copiedBuffer(error.getMessage(), CharsetUtil.UTF_8));
+      response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=UTF-8");
+      ChannelPromise promise = ctx.newPromise();
+      promise.addListener(new GenericFutureListener<ChannelFuture>() {
+
+        @Override
+        public void operationComplete(ChannelFuture future) throws Exception {
+          if (!future.isSuccess()) {
+            throw new ContainerException(error);
+          }
+        }
+      });
+
+      ctx.writeAndFlush(response, promise);
+    }
+
+    @Override
+    public void setSuspendTimeout(long timeOut, TimeUnit timeUnit) throws IllegalStateException {
+      throw new UnsupportedOperationException("setSuspendTimeout is not supported on this container.");
+    }
+
+    @Override
+    public boolean suspend(long timeOut, TimeUnit timeUnit, TimeoutHandler timeoutHandler) {
+      throw new UnsupportedOperationException("suspend is not supported on this container.");
+    }
+
+    @Override
+    public OutputStream writeResponseStatusAndHeaders(long contentLength, ContainerResponse context)
+        throws ContainerException {
+      MultivaluedMap<String, String> responseHeaders = context.getStringHeaders();
+      HttpHeaders nettyHeaders = response.headers();
+
+      for (Entry<String, List<String>> entry: responseHeaders.entrySet()) {
+        nettyHeaders.add(entry.getKey(), entry.getValue());
+      }
+
+      int status = context.getStatus();
+
+      response.setStatus(HttpResponseStatus.valueOf(status));
+      return new ByteBufOutputStream(response.content());
+    }
+
+    private void sendLastHttpContent() {
+      ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT)
+            .addListener(ChannelFutureListener.CLOSE);
+    }
+    
+    private void releaseConnection() {
+      if (closed.compareAndSet(false, true)) {
+        String warnMessage = "ResponseWriter did not be commited.";
+        LOG.warn(warnMessage);
+        failure(new IllegalStateException(warnMessage));
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProvider.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProvider.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProvider.java
new file mode 100644
index 0000000..7481cfb
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestHandlerContainerProvider.java
@@ -0,0 +1,42 @@
+/**
+ * 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.ws.rs.netty;
+
+import io.netty.channel.ChannelHandler;
+
+import javax.ws.rs.ProcessingException;
+
+import org.glassfish.jersey.server.ApplicationHandler;
+import org.glassfish.jersey.server.spi.ContainerProvider;
+
+/**
+ * Container Provider for NettyRestHandlerContainer
+ */
+public final class NettyRestHandlerContainerProvider implements ContainerProvider {
+
+  @Override
+  public <T> T createContainer(Class<T> type, ApplicationHandler application) throws ProcessingException {
+    if (type != NettyRestHandlerContainer.class && 
+        (type == null || !ChannelHandler.class.isAssignableFrom(type))) {
+      return null;
+    }
+    return type.cast(new NettyRestHandlerContainer(application));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServer.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServer.java
new file mode 100644
index 0000000..f7fe148
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServer.java
@@ -0,0 +1,67 @@
+/**
+ * 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.ws.rs.netty;
+
+import io.netty.channel.ChannelHandler;
+import java.net.InetSocketAddress;
+
+import org.apache.tajo.rpc.NettyServerBase;
+
+/**
+ * JAX-RS Http Server on Netty implementation.
+ */
+public class NettyRestServer extends NettyServerBase {
+  
+  private ChannelHandler handler;
+  private int workerCount;
+
+  public NettyRestServer(InetSocketAddress address, int workerCount) {
+    this("NettyRestService", address, workerCount);
+  }
+
+  public NettyRestServer(String serviceName, InetSocketAddress address, int workerCount) {
+    super(serviceName, address);
+    
+    this.workerCount = workerCount;
+  }
+
+  public ChannelHandler getHandler() {
+    return handler;
+  }
+
+  public void setHandler(ChannelHandler handler) {
+    this.handler = handler;
+  }
+
+  /**
+   * Bind desired port and start network service. Before starting network service, {@link NettyRestServer}
+   * will initialize its configuration.
+   * 
+   */
+  @Override
+  public void start() {
+    if (handler == null) {
+      throw new IllegalStateException("ChannelHandler is null.");
+    }
+    
+    super.init(new NettyRestChannelInitializer(handler), workerCount);
+    super.start();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerFactory.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerFactory.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerFactory.java
new file mode 100644
index 0000000..5d2eea1
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerFactory.java
@@ -0,0 +1,89 @@
+/**
+ * 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.ws.rs.netty;
+
+import java.net.InetSocketAddress;
+import java.net.URI;
+
+import org.glassfish.hk2.api.ServiceLocator;
+import org.glassfish.jersey.server.ResourceConfig;
+
+/**
+ * Factory class for creating {@link NettyRestServer} instances
+ */
+public final class NettyRestServerFactory {
+
+  public static NettyRestServer createNettyRestServer(URI uri, ResourceConfig configuration, int workerCount) {
+    return createNettyRestServer(uri, new NettyRestHandlerContainer(configuration), workerCount, true);
+  }
+
+  public static NettyRestServer createNettyRestServer(URI uri, ResourceConfig configuration, int workerCount,
+      boolean start) {
+    return createNettyRestServer(uri, new NettyRestHandlerContainer(configuration), workerCount, start);
+  }
+
+  public static NettyRestServer createNettyRestServer(URI uri, ResourceConfig configuration,
+      ServiceLocator parentLocator, int workerCount) {
+    return createNettyRestServer(uri, new NettyRestHandlerContainer(configuration, parentLocator), workerCount, true);
+  }
+
+  public static NettyRestServer createNettyRestServer(URI uri, ResourceConfig configuration,
+      ServiceLocator parentLocator, int workerCount, boolean start) {
+    return createNettyRestServer(uri, new NettyRestHandlerContainer(configuration, parentLocator), workerCount, start);
+  }
+
+  /**
+   * Creates {@link NettyRestServer} instances with JAX-RS application.
+   * 
+   * @param uri
+   * @param handler
+   * @param start
+   * @return
+   */
+  private static NettyRestServer createNettyRestServer(URI uri, NettyRestHandlerContainer handler, int workerCount,
+      boolean start) {
+    if (uri == null) {
+      throw new IllegalArgumentException("uri is null.");
+    }
+
+    String schemeString = uri.getScheme();
+    if (!schemeString.equalsIgnoreCase("http") && !schemeString.equalsIgnoreCase("https")) {
+      throw new IllegalArgumentException("scheme of this uri (" + uri.toString() + ") should be http or https.");
+    }
+
+    int port = uri.getPort();
+    if (port == -1) {
+      throw new IllegalArgumentException("Port number should be provided.");
+    }
+
+    handler.setRootPath(uri.getPath());
+
+    InetSocketAddress bindAddress = new InetSocketAddress(port);
+    NettyRestServer nettyRestServer = new NettyRestServer("Tajo-REST", bindAddress, workerCount);
+
+    nettyRestServer.setHandler(handler);
+    nettyRestServer.addListener(new NettyRestServerListener(handler));
+
+    if (start) {
+      nettyRestServer.start();
+    }
+
+    return nettyRestServer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerListener.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerListener.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerListener.java
new file mode 100644
index 0000000..ecd5bb0
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/NettyRestServerListener.java
@@ -0,0 +1,72 @@
+/**
+ * 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.ws.rs.netty;
+
+import org.apache.tajo.rpc.RpcEventListener;
+import org.glassfish.jersey.server.ApplicationHandler;
+import org.glassfish.jersey.server.internal.ConfigHelper;
+import org.glassfish.jersey.server.spi.Container;
+import org.glassfish.jersey.server.spi.ContainerLifecycleListener;
+
+/**
+ * Event subscriber for netty rest service.
+ */
+public class NettyRestServerListener implements RpcEventListener {
+  
+  private Container container;
+  
+  public NettyRestServerListener(Container container) {
+    this.container = container;
+  }
+
+  @Override
+  public void onAfterInit(Object obj) {
+    
+  }
+
+  @Override
+  public void onAfterShutdown(Object obj) {
+    ApplicationHandler applicationHandler = new ApplicationHandler(container.getConfiguration());
+    ContainerLifecycleListener lifecycleListener = ConfigHelper.getContainerLifecycleListener(applicationHandler);
+    lifecycleListener.onShutdown(container);
+  }
+
+  @Override
+  public void onAfterStart(Object obj) {
+    ApplicationHandler applicationHandler = new ApplicationHandler(container.getConfiguration());
+    ContainerLifecycleListener lifecycleListener = ConfigHelper.getContainerLifecycleListener(applicationHandler);
+    lifecycleListener.onStartup(container);
+  }
+
+  @Override
+  public void onBeforeInit(Object obj) {
+    
+  }
+
+  @Override
+  public void onBeforeShutdown(Object obj) {
+    
+  }
+
+  @Override
+  public void onBeforeStart(Object obj) {
+    
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonFeature.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonFeature.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonFeature.java
new file mode 100644
index 0000000..26086d4
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonFeature.java
@@ -0,0 +1,34 @@
+/**
+ * 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.ws.rs.netty.gson;
+
+import javax.ws.rs.core.Feature;
+import javax.ws.rs.core.FeatureContext;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.MessageBodyWriter;
+
+public class GsonFeature implements Feature {
+
+  @Override
+  public boolean configure(FeatureContext featureContext) {
+    featureContext.register(GsonReader.class, MessageBodyReader.class);
+    featureContext.register(GsonWriter.class, MessageBodyWriter.class);
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonReader.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonReader.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonReader.java
new file mode 100644
index 0000000..4d6e440
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonReader.java
@@ -0,0 +1,52 @@
+/**
+ * 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.ws.rs.netty.gson;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import java.io.*;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+
+/**
+ * Custom message body reader with Gson feature.
+ */
+@Consumes(MediaType.APPLICATION_JSON)
+public class GsonReader<T> implements MessageBodyReader<T> {
+
+  @Override
+  public boolean isReadable(Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return GsonUtil.isJsonType(mediaType);
+  }
+
+  @Override
+  public T readFrom(Class<T> aClass, Type type, Annotation[] annotations, MediaType mediaType,
+                    MultivaluedMap<String, String> multivaluedMap, InputStream inputStream)
+      throws IOException, WebApplicationException {
+    Gson gson = new GsonBuilder().create();
+    Reader reader = new BufferedReader(new InputStreamReader(inputStream));
+    return gson.fromJson(reader, type);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonUtil.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonUtil.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonUtil.java
new file mode 100644
index 0000000..f16cb96
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonUtil.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.ws.rs.netty.gson;
+
+import javax.ws.rs.core.MediaType;
+
+public class GsonUtil {
+
+  public static boolean isJsonType(MediaType mediaType) {
+    if (mediaType != null) {
+      String subType = mediaType.getSubtype();
+      return "json".equalsIgnoreCase(subType) || subType.endsWith("+json");
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonWriter.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonWriter.java b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonWriter.java
new file mode 100644
index 0000000..d215611
--- /dev/null
+++ b/tajo-rpc/tajo-ws-rs/src/main/java/org/apache/tajo/ws/rs/netty/gson/GsonWriter.java
@@ -0,0 +1,59 @@
+/**
+ * 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.ws.rs.netty.gson;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import java.io.*;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+
+/**
+ * custom message body writer with Gson feature.
+ */
+@Produces(MediaType.APPLICATION_JSON)
+public class GsonWriter<T> implements MessageBodyWriter<T> {
+
+  @Override
+  public boolean isWriteable(Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return GsonUtil.isJsonType(mediaType);
+  }
+
+  @Override
+  public long getSize(T t, Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return 0;
+  }
+
+  @Override
+  public void writeTo(T t, Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType,
+                      MultivaluedMap<String, Object> multivaluedMap, OutputStream outputStream)
+      throws IOException, WebApplicationException {
+    Gson gson = new GsonBuilder().create();
+    Writer writer = new BufferedWriter(new OutputStreamWriter(outputStream));
+
+    gson.toJson(t, type, writer);
+    writer.flush();
+  }
+}


[03/13] tajo git commit: TAJO-1383: Improve broadcast table cache. (jinho)

Posted by ji...@apache.org.
TAJO-1383: Improve broadcast table cache. (jinho)

closes #404


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

Branch: refs/heads/index_support
Commit: e1e38e231867e4f6f953a7ec41f5f9d5ad242580
Parents: 7f05695
Author: Jinho Kim <jh...@apache.org>
Authored: Fri Mar 13 16:55:25 2015 +0900
Committer: Jinho Kim <jh...@apache.org>
Committed: Fri Mar 13 16:55:25 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../engine/planner/physical/HashJoinExec.java   |  51 ++++++--
 .../planner/physical/HashLeftOuterJoinExec.java |  74 ++++++++++-
 .../physical/PartitionMergeScanExec.java        |  17 ++-
 .../tajo/engine/planner/physical/ScanExec.java  |  72 +++++++++++
 .../engine/planner/physical/SeqScanExec.java    | 105 +++-------------
 .../apache/tajo/engine/utils/CacheHolder.java   |  97 +++++++++++++++
 .../apache/tajo/engine/utils/TableCache.java    |  84 +++++++++++++
 .../apache/tajo/engine/utils/TableCacheKey.java |  57 +++++++++
 .../apache/tajo/engine/utils/TupleCache.java    | 122 -------------------
 .../apache/tajo/engine/utils/TupleCacheKey.java |  57 ---------
 .../worker/ExecutionBlockSharedResource.java    |  26 ++++
 .../apache/tajo/worker/TaskAttemptContext.java  |   2 +-
 .../apache/tajo/worker/TaskRunnerManager.java   |   4 +-
 .../apache/tajo/engine/util/TestTableCache.java | 109 +++++++++++++++++
 .../apache/tajo/engine/util/TestTupleCache.java |  89 --------------
 .../plan/serder/LogicalNodeDeserializer.java    |   3 +
 .../tajo/plan/serder/LogicalNodeSerializer.java |   2 +
 tajo-plan/src/main/proto/Plan.proto             |   1 +
 19 files changed, 599 insertions(+), 375 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index e6f7917..84a7571 100644
--- a/CHANGES
+++ b/CHANGES
@@ -9,6 +9,8 @@ Release 0.11.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1383: Improve broadcast table cache. (jinho)
+
     TAJO-1374: Support multi-bytes delimiter for CSV file.
     (Contributed by navis, Committed by jinho)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java
index d475b78..3bdf2d4 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashJoinExec.java
@@ -19,15 +19,18 @@
 package org.apache.tajo.engine.planner.physical;
 
 import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.engine.planner.Projector;
-import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.catalog.SchemaUtil;
+import org.apache.tajo.engine.utils.CacheHolder;
+import org.apache.tajo.engine.utils.TableCacheKey;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.plan.logical.JoinNode;
+import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.FrameTuple;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.ExecutionBlockSharedResource;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
@@ -58,13 +61,14 @@ public class HashJoinExec extends BinaryPhysicalExec {
   // projection
   protected final Projector projector;
 
+  private TableStats cachedRightTableStats;
+
   public HashJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftExec,
       PhysicalExec rightExec) {
     super(context, SchemaUtil.merge(leftExec.getSchema(), rightExec.getSchema()), plan.getOutSchema(),
         leftExec, rightExec);
     this.plan = plan;
     this.joinQual = plan.getJoinQual();
-    this.tupleSlots = new HashMap<Tuple, List<Tuple>>(100000);
 
     // HashJoin only can manage equi join key pairs.
     this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, leftExec.getSchema(),
@@ -151,8 +155,41 @@ public class HashJoinExec extends BinaryPhysicalExec {
   }
 
   protected void loadRightToHashTable() throws IOException {
+    ScanExec scanExec = PhysicalPlanUtil.findExecutor(rightChild, ScanExec.class);
+    if (scanExec.canBroadcast()) {
+      /* If this table can broadcast, all tasks in a node will share the same cache */
+      TableCacheKey key = CacheHolder.BroadcastCacheHolder.getCacheKey(
+          context, scanExec.getCanonicalName(), scanExec.getFragments());
+      loadRightFromCache(key);
+    } else {
+      this.tupleSlots = buildRightToHashTable();
+    }
+
+    first = false;
+  }
+
+  protected void loadRightFromCache(TableCacheKey key) throws IOException {
+    ExecutionBlockSharedResource sharedResource = context.getSharedResource();
+    synchronized (sharedResource.getLock()) {
+      if (sharedResource.hasBroadcastCache(key)) {
+        CacheHolder<Map<Tuple, List<Tuple>>> data = sharedResource.getBroadcastCache(key);
+        this.tupleSlots = data.getData();
+        this.cachedRightTableStats = data.getTableStats();
+      } else {
+        CacheHolder.BroadcastCacheHolder holder =
+            new CacheHolder.BroadcastCacheHolder(buildRightToHashTable(), rightChild.getInputStats(), null);
+        sharedResource.addBroadcastCache(key, holder);
+        CacheHolder<Map<Tuple, List<Tuple>>> data = sharedResource.getBroadcastCache(key);
+        this.tupleSlots = data.getData();
+        this.cachedRightTableStats = data.getTableStats();
+      }
+    }
+  }
+
+  private Map<Tuple, List<Tuple>> buildRightToHashTable() throws IOException {
     Tuple tuple;
     Tuple keyTuple;
+    Map<Tuple, List<Tuple>> map = new HashMap<Tuple, List<Tuple>>(100000);
 
     while (!context.isStopped() && (tuple = rightChild.next()) != null) {
       keyTuple = new VTuple(joinKeyPairs.size());
@@ -160,18 +197,18 @@ public class HashJoinExec extends BinaryPhysicalExec {
         keyTuple.put(i, tuple.get(rightKeyList[i]));
       }
 
-      List<Tuple> newValue = tupleSlots.get(keyTuple);
+      List<Tuple> newValue = map.get(keyTuple);
 
       if (newValue != null) {
         newValue.add(tuple);
       } else {
         newValue = new ArrayList<Tuple>();
         newValue.add(tuple);
-        tupleSlots.put(keyTuple, newValue);
+        map.put(keyTuple, newValue);
       }
     }
 
-    first = false;
+    return map;
   }
 
   @Override
@@ -219,7 +256,7 @@ public class HashJoinExec extends BinaryPhysicalExec {
       inputStats.setNumRows(leftInputStats.getNumRows());
     }
 
-    TableStats rightInputStats = rightChild.getInputStats();
+    TableStats rightInputStats = cachedRightTableStats == null ? rightChild.getInputStats() : cachedRightTableStats;
     if (rightInputStats != null) {
       inputStats.setNumBytes(inputStats.getNumBytes() + rightInputStats.getNumBytes());
       inputStats.setReadBytes(inputStats.getReadBytes() + rightInputStats.getReadBytes());

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java
index 233ef92..e78cb20 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java
@@ -22,7 +22,10 @@ import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.utils.CacheHolder;
+import org.apache.tajo.engine.utils.TableCacheKey;
 import org.apache.tajo.engine.utils.TupleUtil;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.catalog.SchemaUtil;
@@ -33,6 +36,7 @@ import org.apache.tajo.plan.logical.JoinNode;
 import org.apache.tajo.storage.FrameTuple;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.worker.ExecutionBlockSharedResource;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
@@ -66,6 +70,7 @@ public class HashLeftOuterJoinExec extends BinaryPhysicalExec {
   protected Projector projector;
 
   private int rightNumCols;
+  private TableStats cachedRightTableStats;
   private static final Log LOG = LogFactory.getLog(HashLeftOuterJoinExec.class);
 
   public HashLeftOuterJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec leftChild,
@@ -91,8 +96,6 @@ public class HashLeftOuterJoinExec extends BinaryPhysicalExec {
       this.joinFilter = null;
     }
 
-    this.tupleSlots = new HashMap<Tuple, List<Tuple>>(10000);
-
     // HashJoin only can manage equi join key pairs.
     this.joinKeyPairs = PlannerUtil.getJoinKeyPairs(joinQual, leftChild.getSchema(),
         rightChild.getSchema(), false);
@@ -201,8 +204,41 @@ public class HashLeftOuterJoinExec extends BinaryPhysicalExec {
   }
 
   protected void loadRightToHashTable() throws IOException {
+    ScanExec scanExec = PhysicalPlanUtil.findExecutor(rightChild, ScanExec.class);
+    if (scanExec.canBroadcast()) {
+      /* If this table can broadcast, all tasks in a node will share the same cache */
+      TableCacheKey key = CacheHolder.BroadcastCacheHolder.getCacheKey(
+          context, scanExec.getCanonicalName(), scanExec.getFragments());
+      loadRightFromCache(key);
+    } else {
+      this.tupleSlots = buildRightToHashTable();
+    }
+
+    first = false;
+  }
+
+  protected void loadRightFromCache(TableCacheKey key) throws IOException {
+    ExecutionBlockSharedResource sharedResource = context.getSharedResource();
+    synchronized (sharedResource.getLock()) {
+      if (sharedResource.hasBroadcastCache(key)) {
+        CacheHolder<Map<Tuple, List<Tuple>>> data = sharedResource.getBroadcastCache(key);
+        this.tupleSlots = data.getData();
+        this.cachedRightTableStats = data.getTableStats();
+      } else {
+        CacheHolder.BroadcastCacheHolder holder =
+            new CacheHolder.BroadcastCacheHolder(buildRightToHashTable(), rightChild.getInputStats(), null);
+        sharedResource.addBroadcastCache(key, holder);
+        CacheHolder<Map<Tuple, List<Tuple>>> data = sharedResource.getBroadcastCache(key);
+        this.tupleSlots = data.getData();
+        this.cachedRightTableStats = data.getTableStats();
+      }
+    }
+  }
+
+  private Map<Tuple, List<Tuple>> buildRightToHashTable() throws IOException {
     Tuple tuple;
     Tuple keyTuple;
+    Map<Tuple, List<Tuple>> map = new HashMap<Tuple, List<Tuple>>(100000);
 
     while (!context.isStopped() && (tuple = rightChild.next()) != null) {
       keyTuple = new VTuple(joinKeyPairs.size());
@@ -210,16 +246,18 @@ public class HashLeftOuterJoinExec extends BinaryPhysicalExec {
         keyTuple.put(i, tuple.get(rightKeyList[i]));
       }
 
-      List<Tuple> newValue = tupleSlots.get(keyTuple);
+      List<Tuple> newValue = map.get(keyTuple);
+
       if (newValue != null) {
         newValue.add(tuple);
       } else {
         newValue = new ArrayList<Tuple>();
         newValue.add(tuple);
-        tupleSlots.put(keyTuple, newValue);
+        map.put(keyTuple, newValue);
       }
     }
-    first = false;
+
+    return map;
   }
 
   @Override
@@ -250,5 +288,31 @@ public class HashLeftOuterJoinExec extends BinaryPhysicalExec {
   public JoinNode getPlan() {
     return this.plan;
   }
+
+  @Override
+  public TableStats getInputStats() {
+    if (leftChild == null) {
+      return inputStats;
+    }
+    TableStats leftInputStats = leftChild.getInputStats();
+    inputStats.setNumBytes(0);
+    inputStats.setReadBytes(0);
+    inputStats.setNumRows(0);
+
+    if (leftInputStats != null) {
+      inputStats.setNumBytes(leftInputStats.getNumBytes());
+      inputStats.setReadBytes(leftInputStats.getReadBytes());
+      inputStats.setNumRows(leftInputStats.getNumRows());
+    }
+
+    TableStats rightInputStats = cachedRightTableStats == null ? rightChild.getInputStats() : cachedRightTableStats;
+    if (rightInputStats != null) {
+      inputStats.setNumBytes(inputStats.getNumBytes() + rightInputStats.getNumBytes());
+      inputStats.setReadBytes(inputStats.getReadBytes() + rightInputStats.getReadBytes());
+      inputStats.setNumRows(inputStats.getNumRows() + rightInputStats.getNumRows());
+    }
+
+    return inputStats;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
index 5692308..a1eaa48 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
@@ -34,7 +34,7 @@ import java.util.List;
 /**
  * A Scanner that reads multiple partitions
  */
-public class PartitionMergeScanExec extends PhysicalExec {
+public class PartitionMergeScanExec extends ScanExec {
   private final ScanNode plan;
   private SeqScanExec currentScanner = null;
 
@@ -56,14 +56,16 @@ public class PartitionMergeScanExec extends PhysicalExec {
     inputStats = new TableStats();
   }
 
+  @Override
   public void init() throws IOException {
     for (CatalogProtos.FragmentProto fragment : fragments) {
       SeqScanExec scanExec = new SeqScanExec(context, (ScanNode) PlannerUtil.clone(null, plan),
-          new CatalogProtos.FragmentProto[] {fragment});
+          new CatalogProtos.FragmentProto[]{fragment});
       scanners.add(scanExec);
     }
     progress = 0.0f;
     rescan();
+    super.init();
   }
 
   @Override
@@ -112,11 +114,22 @@ public class PartitionMergeScanExec extends PhysicalExec {
     progress = 1.0f;
   }
 
+  @Override
   public String getTableName() {
     return plan.getTableName();
   }
 
   @Override
+  public String getCanonicalName() {
+    return plan.getCanonicalName();
+  }
+
+  @Override
+  public CatalogProtos.FragmentProto[] getFragments() {
+    return fragments;
+  }
+
+  @Override
   public float getProgress() {
     if (iterator != null) {
       float progressSum = 0.0f;

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java
new file mode 100644
index 0000000..86874ba
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanExec.java
@@ -0,0 +1,72 @@
+/**
+ * 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.engine.planner.physical;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.List;
+
+public abstract class ScanExec extends PhysicalExec {
+
+  /* if this is a broadcasted table or not  */
+  private boolean canBroadcast;
+
+  public ScanExec(TaskAttemptContext context, Schema inSchema, Schema outSchema) {
+    super(context, inSchema, outSchema);
+  }
+
+  public abstract String getTableName();
+
+  public abstract String getCanonicalName();
+
+  public abstract CatalogProtos.FragmentProto[] getFragments();
+
+  @Override
+  public void init() throws IOException {
+    canBroadcast = checkIfBroadcast();
+
+    super.init();
+  }
+
+  public boolean canBroadcast() {
+    return canBroadcast;
+  }
+
+  /* check if this scan is broadcasted */
+  private boolean checkIfBroadcast() {
+    Enforcer enforcer = context.getEnforcer();
+
+    if (enforcer != null && enforcer.hasEnforceProperty(TajoWorkerProtocol.EnforceProperty.EnforceType.BROADCAST)) {
+      List<TajoWorkerProtocol.EnforceProperty> properties =
+          enforcer.getEnforceProperties(TajoWorkerProtocol.EnforceProperty.EnforceType.BROADCAST);
+
+      for (TajoWorkerProtocol.EnforceProperty property : properties) {
+        if (getCanonicalName().equals(property.getBroadcast().getTableName())) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
index c62027d..1078c80 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
@@ -21,17 +21,14 @@ package org.apache.tajo.engine.planner.physical;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.engine.codegen.CompilationError;
 import org.apache.tajo.engine.planner.Projector;
-import org.apache.tajo.engine.utils.TupleCache;
-import org.apache.tajo.engine.utils.TupleCacheKey;
-import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.plan.Target;
 import org.apache.tajo.plan.expr.ConstEval;
 import org.apache.tajo.plan.expr.EvalNode;
@@ -42,18 +39,16 @@ import org.apache.tajo.plan.rewrite.rules.PartitionedTableRewriter;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
 
-public class SeqScanExec extends PhysicalExec {
+public class SeqScanExec extends ScanExec {
   private ScanNode plan;
 
   private Scanner scanner = null;
@@ -66,10 +61,6 @@ public class SeqScanExec extends PhysicalExec {
 
   private TableStats inputStats;
 
-  private TupleCacheKey cacheKey;
-
-  private boolean cacheRead = false;
-
   public SeqScanExec(TaskAttemptContext context, ScanNode plan,
                      CatalogProtos.FragmentProto [] fragments) throws IOException {
     super(context, plan.getInSchema(), plan.getOutSchema());
@@ -78,21 +69,6 @@ public class SeqScanExec extends PhysicalExec {
     this.qual = plan.getQual();
     this.fragments = fragments;
 
-    if (plan.isBroadcastTable()) {
-      String pathNameKey = "";
-      if (fragments != null) {
-        StringBuilder stringBuilder = new StringBuilder();
-        for (FragmentProto f : fragments) {
-          Fragment fragement = FragmentConvertor.convert(context.getConf(), f);
-          stringBuilder.append(fragement.getKey());
-        }
-        pathNameKey = stringBuilder.toString();
-      }
-
-      cacheKey = new TupleCacheKey(
-          context.getTaskId().getTaskId().getExecutionBlockId().toString(), plan.getTableName(), pathNameKey);
-    }
-
     if (fragments != null
         && plan.getTableDesc().hasPartition()
         && plan.getTableDesc().getPartitionMethod().getPartitionType() == CatalogProtos.PartitionType.COLUMN) {
@@ -153,6 +129,7 @@ public class SeqScanExec extends PhysicalExec {
     }
   }
 
+  @Override
   public void init() throws IOException {
     Schema projected;
 
@@ -177,33 +154,7 @@ public class SeqScanExec extends PhysicalExec {
       projected = outSchema;
     }
 
-    if (cacheKey != null) {
-      TupleCache tupleCache = TupleCache.getInstance();
-      if (tupleCache.isBroadcastCacheReady(cacheKey)) {
-        openCacheScanner();
-      } else {
-        if (TupleCache.getInstance().lockBroadcastScan(cacheKey)) {
-          scanAndAddCache(projected);
-          openCacheScanner();
-        } else {
-          Object lockMonitor = tupleCache.getLockMonitor();
-          synchronized (lockMonitor) {
-            try {
-              lockMonitor.wait(20 * 1000);
-            } catch (InterruptedException e) {
-            }
-          }
-          if (tupleCache.isBroadcastCacheReady(cacheKey)) {
-            openCacheScanner();
-          } else {
-            initScanner(projected);
-          }
-        }
-      }
-    } else {
-      initScanner(projected);
-    }
-
+    initScanner(projected);
     super.init();
   }
 
@@ -216,7 +167,7 @@ public class SeqScanExec extends PhysicalExec {
 
   private void initScanner(Schema projected) throws IOException {
     this.projector = new Projector(context, inSchema, outSchema, plan.getTargets());
-    TableMeta meta = null;
+    TableMeta meta;
     try {
       meta = (TableMeta) plan.getTableDesc().getMeta().clone();
     } catch (CloneNotSupportedException e) {
@@ -241,35 +192,6 @@ public class SeqScanExec extends PhysicalExec {
     }
   }
 
-  private void openCacheScanner() throws IOException {
-    Scanner cacheScanner = TupleCache.getInstance().openCacheScanner(cacheKey, plan.getPhysicalSchema());
-    if (cacheScanner != null) {
-      scanner = cacheScanner;
-      cacheRead = true;
-    }
-  }
-
-  private void scanAndAddCache(Schema projected) throws IOException {
-    initScanner(projected);
-
-    List<Tuple> broadcastTupleCacheList = new ArrayList<Tuple>();
-    while (!context.isStopped()) {
-      Tuple tuple = next();
-      if (tuple != null) {
-        broadcastTupleCacheList.add(tuple);
-      } else {
-        break;
-      }
-    }
-
-    if (scanner != null) {
-      scanner.close();
-      scanner = null;
-    }
-
-    TupleCache.getInstance().addBroadcastCache(cacheKey, broadcastTupleCacheList);
-  }
-
   @Override
   public Tuple next() throws IOException {
     if (fragments == null) {
@@ -281,9 +203,6 @@ public class SeqScanExec extends PhysicalExec {
 
     if (!plan.hasQual()) {
       if ((tuple = scanner.next()) != null) {
-        if (cacheRead) {
-          return tuple;
-        }
         projector.eval(tuple, outTuple);
         outTuple.setOffset(tuple.getOffset());
         return outTuple;
@@ -292,9 +211,6 @@ public class SeqScanExec extends PhysicalExec {
       }
     } else {
       while ((tuple = scanner.next()) != null) {
-        if (cacheRead) {
-          return tuple;
-        }
         if (qual.eval(inSchema, tuple).isTrue()) {
           projector.eval(tuple, outTuple);
           return outTuple;
@@ -328,11 +244,22 @@ public class SeqScanExec extends PhysicalExec {
     projector = null;
   }
 
+  @Override
   public String getTableName() {
     return plan.getTableName();
   }
 
   @Override
+  public String getCanonicalName() {
+    return plan.getCanonicalName();
+  }
+
+  @Override
+  public CatalogProtos.FragmentProto[] getFragments() {
+    return fragments;
+  }
+
+  @Override
   public float getProgress() {
     if (scanner == null) {
       return 1.0f;

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java
new file mode 100644
index 0000000..6a5c0bf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/CacheHolder.java
@@ -0,0 +1,97 @@
+/**
+ * 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.engine.utils;
+
+import com.google.common.collect.Maps;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.util.Deallocatable;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+public interface CacheHolder<T> {
+
+  /**
+   * Get a shared data from the TableCache.
+   */
+  T getData();
+
+  /**
+   * Get a shared table stats from the TableCache.
+   */
+  TableStats getTableStats();
+
+  /**
+   * Release a cache to the memory.
+   *
+   */
+  void release();
+
+  /**
+   * This is a cache-holder for a join table
+   * It will release when execution block is finished
+   */
+  public static class BroadcastCacheHolder implements CacheHolder<Map<Tuple, List<Tuple>>> {
+    private Map<Tuple, List<Tuple>> data;
+    private Deallocatable rowBlock;
+    private TableStats tableStats;
+
+    public BroadcastCacheHolder(Map<Tuple, List<Tuple>> data, TableStats tableStats, Deallocatable rowBlock){
+      this.data = data;
+      this.tableStats = tableStats;
+      this.rowBlock = rowBlock;
+    }
+
+    @Override
+    public Map<Tuple, List<Tuple>> getData() {
+      return Maps.newHashMap(data);
+    }
+
+    @Override
+    public TableStats getTableStats(){
+      return tableStats;
+    }
+
+    @Override
+    public void release() {
+      if(rowBlock != null) rowBlock.release();
+    }
+
+    public static TableCacheKey getCacheKey(TaskAttemptContext ctx, String canonicalName,
+                                                 CatalogProtos.FragmentProto[] fragments) throws IOException {
+      String pathNameKey = "";
+      if (fragments != null) {
+        StringBuilder stringBuilder = new StringBuilder();
+        for (CatalogProtos.FragmentProto f : fragments) {
+          Fragment fragement = FragmentConvertor.convert(ctx.getConf(), f);
+          stringBuilder.append(fragement.getKey());
+        }
+        pathNameKey = stringBuilder.toString();
+      }
+
+      return new TableCacheKey(ctx.getTaskId().getTaskId().getExecutionBlockId().toString(), canonicalName, pathNameKey);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCache.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCache.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCache.java
new file mode 100644
index 0000000..f2a2217
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCache.java
@@ -0,0 +1,84 @@
+/**
+ * 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.engine.utils;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.ExecutionBlockId;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This is a simple TableCache which just added CacheHolder as needed.
+ */
+ public class TableCache {
+  public static final Log LOG = LogFactory.getLog(TableCache.class);
+
+  private static TableCache instance;
+  private Map<TableCacheKey, CacheHolder<?>> cacheMap = Maps.newHashMap();
+
+  private TableCache() {
+  }
+
+  public static synchronized TableCache getInstance() {
+    if (instance == null) {
+      instance = new TableCache();
+    }
+    return instance;
+  }
+
+  public synchronized void releaseCache(ExecutionBlockId ebId) {
+    if (ebId == null) {
+      return;
+    }
+
+    List<TableCacheKey> keys = getCacheKeyByExecutionBlockId(ebId);
+
+    for (TableCacheKey cacheKey: keys) {
+      cacheMap.remove(cacheKey).release();
+      LOG.info("Removed Broadcast Table Cache: " + cacheKey.getTableName() + " EbId: " + cacheKey.ebId);
+    }
+  }
+
+  public synchronized List<TableCacheKey> getCacheKeyByExecutionBlockId(ExecutionBlockId ebId) {
+    List<TableCacheKey> keys = Lists.newArrayList();
+    for (TableCacheKey eachKey : cacheMap.keySet()) {
+      if (eachKey.ebId.equals(ebId.toString())) {
+        keys.add(eachKey);
+      }
+    }
+    return keys;
+  }
+
+  public synchronized void addCache(TableCacheKey cacheKey,  CacheHolder<?> cacheData) {
+    cacheMap.put(cacheKey, cacheData);
+    LOG.info("Added Broadcast Table Cache: " + cacheKey.getTableName() + " EbId: " + cacheKey.ebId);
+  }
+
+  public synchronized boolean hasCache(TableCacheKey cacheKey) {
+    return cacheMap.containsKey(cacheKey);
+  }
+
+  public synchronized CacheHolder<?> getCache(TableCacheKey cacheKey) {
+    return cacheMap.get(cacheKey);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCacheKey.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCacheKey.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCacheKey.java
new file mode 100644
index 0000000..81a4b58
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TableCacheKey.java
@@ -0,0 +1,57 @@
+/**
+ * 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.engine.utils;
+
+public class TableCacheKey {
+  String ebId;
+  String tableName;
+  String pathName;
+
+  public TableCacheKey(String ebId, String tableName, String pathName) {
+    this.ebId = ebId;
+    this.tableName = tableName;
+    this.pathName = pathName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    return toString().equals(o.toString());
+  }
+
+  @Override
+  public String toString() {
+    return ebId + "," + tableName + "," + pathName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java
deleted file mode 100644
index 00647b5..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCache.java
+++ /dev/null
@@ -1,122 +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.engine.utils;
-
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class TupleCache {
-  private static TupleCache instance;
-
-  private Map<TupleCacheKey, List<Tuple>> broadcastTupleCacheData
-      = new HashMap<TupleCacheKey, List<Tuple>>();
-  private Map<TupleCacheKey, TupleCacheStatus> broadcastTupleCacheStatus
-      = new HashMap<TupleCacheKey, TupleCacheStatus>();
-
-  private Object lockMonitor = new Object();
-
-  public static enum TupleCacheStatus {
-    STARTED,
-    ENDED
-  };
-
-  private TupleCache() {
-  }
-
-  public static synchronized TupleCache getInstance() {
-    if (instance == null) {
-      instance = new TupleCache();
-    }
-    return instance;
-  }
-
-  public Object getLockMonitor() {
-    return lockMonitor;
-  }
-
-  public void removeBroadcastCache(ExecutionBlockId ebId) {
-    if (ebId == null) {
-      return;
-    }
-    synchronized (lockMonitor) {
-      TupleCacheKey matchedKey = null;
-      for (TupleCacheKey eachKey: broadcastTupleCacheStatus.keySet()) {
-        if (eachKey.ebId.equals(ebId.toString())) {
-          matchedKey = eachKey;
-          break;
-        }
-      }
-      if (matchedKey != null) {
-        broadcastTupleCacheStatus.remove(matchedKey);
-        broadcastTupleCacheData.remove(matchedKey);
-      }
-    }
-  }
-
-  public void addBroadcastCache(TupleCacheKey cacheKey, List<Tuple> cacheData) {
-    synchronized (lockMonitor) {
-      if (broadcastTupleCacheStatus.containsKey(cacheKey) &&
-          broadcastTupleCacheStatus.get(cacheKey) == TupleCacheStatus.ENDED) {
-        return;
-      }
-      broadcastTupleCacheData.put(cacheKey, cacheData);
-      broadcastTupleCacheStatus.put(cacheKey, TupleCacheStatus.ENDED);
-      lockMonitor.notifyAll();
-    }
-  }
-
-  public boolean lockBroadcastScan(TupleCacheKey cacheKey) {
-    synchronized (lockMonitor) {
-      if (broadcastTupleCacheStatus.containsKey(cacheKey)) {
-        return false;
-      } else {
-        broadcastTupleCacheStatus.put(cacheKey, TupleCacheStatus.STARTED);
-        return true;
-      }
-    }
-  }
-
-  public boolean isBroadcastCacheReady(TupleCacheKey cacheKey) {
-    synchronized (lockMonitor) {
-      if (!broadcastTupleCacheStatus.containsKey(cacheKey)) {
-        return false;
-      }
-      return broadcastTupleCacheStatus.get(cacheKey) == TupleCacheStatus.ENDED;
-    }
-  }
-
-  public TupleCacheScanner openCacheScanner(TupleCacheKey cacheKey, Schema schema) throws IOException {
-    synchronized (lockMonitor) {
-      List<Tuple> cacheData = broadcastTupleCacheData.get(cacheKey);
-      if (cacheData != null) {
-        TupleCacheScanner scanner = new TupleCacheScanner(cacheData, schema);
-        scanner.init();
-        return scanner;
-      } else {
-        return null;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java
deleted file mode 100644
index 1cb01c2..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleCacheKey.java
+++ /dev/null
@@ -1,57 +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.engine.utils;
-
-public class TupleCacheKey {
-  String ebId;
-  String tableName;
-  String pathName;
-
-  public TupleCacheKey(String ebId, String tableName, String pathName) {
-    this.ebId = ebId;
-    this.tableName = tableName;
-    this.pathName = pathName;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  @Override
-  public int hashCode() {
-    return toString().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    return toString().equals(o.toString());
-  }
-
-  @Override
-  public String toString() {
-    return ebId + "," + tableName + "," + pathName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java
index b193b24..494fd7f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockSharedResource.java
@@ -21,12 +21,16 @@ package org.apache.tajo.worker;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.engine.codegen.ExecutorPreCompiler;
 import org.apache.tajo.engine.codegen.TajoClassLoader;
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.engine.utils.CacheHolder;
+import org.apache.tajo.engine.utils.TableCache;
+import org.apache.tajo.engine.utils.TableCacheKey;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.plan.logical.LogicalNode;
@@ -38,6 +42,7 @@ public class ExecutionBlockSharedResource {
   private static Log LOG = LogFactory.getLog(ExecutionBlockSharedResource.class);
   private AtomicBoolean initializing = new AtomicBoolean(false);
   private volatile Boolean resourceInitSuccess = Boolean.valueOf(false);
+  private final Object lock = new Object();
 
   // Query
   private QueryContext context;
@@ -108,6 +113,27 @@ public class ExecutionBlockSharedResource {
     }
   }
 
+  /* This is guarantee a lock for a ExecutionBlock */
+  public synchronized Object getLock() {
+    return lock;
+  }
+
+  public boolean hasBroadcastCache(TableCacheKey key) {
+    return TableCache.getInstance().hasCache(key);
+  }
+
+  public <T extends Object> CacheHolder<T> getBroadcastCache(TableCacheKey key) {
+    return (CacheHolder<T>) TableCache.getInstance().getCache(key);
+  }
+
+  public void addBroadcastCache(TableCacheKey cacheKey,  CacheHolder<?> cacheData) {
+    TableCache.getInstance().addCache(cacheKey, cacheData);
+  }
+
+  public void releaseBroadcastCache(ExecutionBlockId id) {
+    TableCache.getInstance().releaseCache(id);
+  }
+
   public void release() {
     compilationContext = null;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
index 50cd20a..706e9b8 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
@@ -24,8 +24,8 @@ import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
-import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.TajoProtos.TaskAttemptState;
+import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
index 3f4a1b8..a375a31 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
@@ -20,7 +20,6 @@ package org.apache.tajo.worker;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -30,7 +29,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.utils.TupleCache;
 import org.apache.tajo.worker.event.TaskRunnerEvent;
 import org.apache.tajo.worker.event.TaskRunnerStartEvent;
 import org.apache.tajo.worker.event.TaskRunnerStopEvent;
@@ -184,7 +182,7 @@ public class TaskRunnerManager extends CompositeService implements EventHandler<
       ExecutionBlockContext executionBlockContext =  executionBlockContextMap.remove(event.getExecutionBlockId());
       if(executionBlockContext != null){
         try {
-          TupleCache.getInstance().removeBroadcastCache(event.getExecutionBlockId());
+          executionBlockContext.getSharedResource().releaseBroadcastCache(event.getExecutionBlockId());
           executionBlockContext.reportExecutionBlock(event.getExecutionBlockId());
           workerContext.getHashShuffleAppenderManager().close(event.getExecutionBlockId());
           workerContext.getTaskHistoryWriter().flushTaskHistories();

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTableCache.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTableCache.java b/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTableCache.java
new file mode 100644
index 0000000..f10f2a1
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTableCache.java
@@ -0,0 +1,109 @@
+/**
+ * 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.engine.util;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.engine.utils.CacheHolder;
+import org.apache.tajo.engine.utils.TableCache;
+import org.apache.tajo.engine.utils.TableCacheKey;
+import org.apache.tajo.worker.ExecutionBlockSharedResource;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TestTableCache {
+
+  @Test
+  public void testBroadcastTableCache() throws Exception {
+
+    ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(
+        QueryIdFactory.newQueryId(System.currentTimeMillis(), 0));
+
+    final TableCacheKey key = new TableCacheKey(ebId.toString(), "testBroadcastTableCache", "path");
+    final ExecutionBlockSharedResource resource = new ExecutionBlockSharedResource();
+
+    final int parallelCount = 30;
+    ExecutorService executor = Executors.newFixedThreadPool(parallelCount);
+    List<Future<CacheHolder<Long>>> tasks = new ArrayList<Future<CacheHolder<Long>>>();
+    for (int i = 0; i < parallelCount; i++) {
+      tasks.add(executor.submit(createTask(key, resource)));
+    }
+
+    long expected = tasks.get(0).get().getData().longValue();
+
+    for (Future<CacheHolder<Long>> future : tasks) {
+      assertEquals(expected, future.get().getData().longValue());
+    }
+
+    resource.releaseBroadcastCache(ebId);
+    assertFalse(resource.hasBroadcastCache(key));
+    executor.shutdown();
+  }
+
+  private Callable<CacheHolder<Long>> createTask(final TableCacheKey key, final ExecutionBlockSharedResource resource) {
+    return new Callable<CacheHolder<Long>>() {
+      @Override
+      public CacheHolder<Long> call() throws Exception {
+        CacheHolder<Long> result;
+        synchronized (resource.getLock()) {
+          if (!TableCache.getInstance().hasCache(key)) {
+            final long nanoTime = System.nanoTime();
+            final TableStats tableStats = new TableStats();
+            tableStats.setNumRows(100);
+            tableStats.setNumBytes(1000);
+
+            final CacheHolder<Long> cacheHolder = new CacheHolder<Long>() {
+
+              @Override
+              public Long getData() {
+                return nanoTime;
+              }
+
+              @Override
+              public TableStats getTableStats() {
+                return tableStats;
+              }
+
+              @Override
+              public void release() {
+
+              }
+            };
+
+            resource.addBroadcastCache(key, cacheHolder);
+          }
+        }
+
+        CacheHolder<?> holder = resource.getBroadcastCache(key);
+        result = (CacheHolder<Long>) holder;
+        return result;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTupleCache.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTupleCache.java b/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTupleCache.java
deleted file mode 100644
index 3d2f307..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/engine/util/TestTupleCache.java
+++ /dev/null
@@ -1,89 +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.engine.util;
-
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.engine.utils.TupleCache;
-import org.apache.tajo.engine.utils.TupleCacheKey;
-import org.apache.tajo.storage.Scanner;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static junit.framework.TestCase.assertEquals;
-import static junit.framework.TestCase.assertFalse;
-import static junit.framework.TestCase.assertTrue;
-import static org.junit.Assert.assertNotNull;
-
-public class TestTupleCache {
-  @Test
-  public void testTupleCcaheBasicFunction() throws Exception {
-    List<Tuple> tupleData = new ArrayList<Tuple>();
-    for (int i = 0; i < 100; i++) {
-      Datum[] datums = new Datum[5];
-      for (int j = 0; j < 5; j++) {
-        datums[j] = new TextDatum(i + "_" + j);
-      }
-      Tuple tuple = new VTuple(datums);
-      tupleData.add(tuple);
-    }
-
-    ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(
-        QueryIdFactory.newQueryId(System.currentTimeMillis(), 0));
-
-    TupleCacheKey cacheKey = new TupleCacheKey(ebId.toString(), "TestTable", "test");
-    TupleCache tupleCache = TupleCache.getInstance();
-
-    assertFalse(tupleCache.isBroadcastCacheReady(cacheKey));
-    assertTrue(tupleCache.lockBroadcastScan(cacheKey));
-    assertFalse(tupleCache.lockBroadcastScan(cacheKey));
-
-    tupleCache.addBroadcastCache(cacheKey, tupleData);
-    assertTrue(tupleCache.isBroadcastCacheReady(cacheKey));
-
-    Scanner scanner = tupleCache.openCacheScanner(cacheKey, null);
-    assertNotNull(scanner);
-
-    int count = 0;
-
-    while (true) {
-      Tuple tuple = scanner.next();
-      if (tuple == null) {
-        break;
-      }
-
-      assertEquals(tupleData.get(count), tuple);
-      count++;
-    }
-
-    assertEquals(tupleData.size(), count);
-
-    tupleCache.removeBroadcastCache(ebId);
-    assertFalse(tupleCache.isBroadcastCacheReady(cacheKey));
-    assertTrue(tupleCache.lockBroadcastScan(cacheKey));
-
-    tupleCache.removeBroadcastCache(ebId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
index 5cbed7e..3387157 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
@@ -416,6 +416,9 @@ public class LogicalNodeDeserializer {
       scan.setQual(EvalNodeDeserializer.deserialize(context, scanProto.getQual()));
     }
 
+    if(scanProto.hasBroadcast()){
+      scan.setBroadcastTable(scanProto.getBroadcast());
+    }
     scan.setInSchema(convertSchema(protoNode.getInSchema()));
     scan.setOutSchema(convertSchema(protoNode.getOutSchema()));
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
index 39a13ba..1bde955 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
@@ -431,6 +431,8 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe
     if (scan.hasQual()) {
       scanBuilder.setQual(EvalNodeSerializer.serialize(scan.getQual()));
     }
+
+    scanBuilder.setBroadcast(scan.isBroadcastTable());
     return scanBuilder;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/e1e38e23/tajo-plan/src/main/proto/Plan.proto
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto
index 3e4f07c..02f52ff 100644
--- a/tajo-plan/src/main/proto/Plan.proto
+++ b/tajo-plan/src/main/proto/Plan.proto
@@ -104,6 +104,7 @@ message ScanNode {
   required bool existTargets = 3;
   repeated Target targets = 4;
   optional EvalNodeTree qual = 5;
+  optional bool broadcast = 6;
 }
 
 message PartitionScanSpec {


[06/13] tajo git commit: TAJO-1381: Support multi-bytes delimiter for Text file

Posted by ji...@apache.org.
TAJO-1381: Support multi-bytes delimiter for Text file

Closes #410

Signed-off-by: Jinho Kim <jh...@apache.org>


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

Branch: refs/heads/index_support
Commit: 82d44af32246c63a32c049292f0a229f16e85768
Parents: 286b956
Author: navis.ryu <na...@apache.org>
Authored: Wed Mar 11 08:49:31 2015 +0900
Committer: Jinho Kim <jh...@apache.org>
Committed: Wed Mar 18 11:47:57 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |  3 ++
 .../tajo/engine/query/TestSelectQuery.java      | 24 +++++++++++
 .../multibytes_delimiter_table3_ddl.sql         |  3 ++
 .../multibytes_delimiter_table4_ddl.sql         |  3 ++
 .../testMultiBytesDelimiter3.sql                |  1 +
 .../testMultiBytesDelimiter4.sql                |  1 +
 .../testMultiBytesDelimiter3.result             |  7 +++
 .../testMultiBytesDelimiter4.result             |  7 +++
 .../java/org/apache/tajo/storage/CSVFile.java   | 11 +++--
 .../tajo/storage/text/CSVLineDeserializer.java  | 14 ++++--
 .../apache/tajo/storage/text/CSVLineSerDe.java  |  5 ++-
 .../tajo/storage/text/CSVLineSerializer.java    |  8 ++--
 .../tajo/storage/text/DelimitedTextFile.java    |  2 +-
 .../tajo/storage/text/FieldSplitProcessor.java  |  8 +---
 .../text/MultiBytesFieldSplitProcessor.java     | 45 ++++++++++++++++++++
 .../tajo/storage/text/TextLineDeserializer.java |  6 +--
 .../apache/tajo/storage/text/TextLineSerDe.java |  3 +-
 .../apache/tajo/storage/TestSplitProcessor.java | 38 +++++++++++++++--
 18 files changed, 162 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 0d7222f..c3f2691 100644
--- a/CHANGES
+++ b/CHANGES
@@ -9,6 +9,9 @@ Release 0.11.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1381: Support multi-bytes delimiter for Text file.
+    (Contributed by navis, Committed by jinho)
+
     TAJO-1391: RpcConnectionPool should check reference counter of connection 
     before close. (Contributed by navis, Committed by jihun)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
index dd93dd1..f7b1382 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
@@ -659,4 +659,28 @@ public class TestSelectQuery extends QueryTestCaseBase {
       executeString("DROP TABLE table2");
     }
   }
+
+  @Test
+  public void testMultiBytesDelimiter3() throws Exception {
+    executeDDL("multibytes_delimiter_table3_ddl.sql", "multibytes_delimiter1");
+    try {
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE table1");
+    }
+  }
+
+  @Test
+  public void testMultiBytesDelimiter4() throws Exception {
+    executeDDL("multibytes_delimiter_table4_ddl.sql", "multibytes_delimiter2");
+    try {
+      ResultSet res = executeQuery();
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE table2");
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table3_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table3_ddl.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table3_ddl.sql
new file mode 100644
index 0000000..8309d11
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table3_ddl.sql
@@ -0,0 +1,3 @@
+create external table table1 (id int, name text, score float, type text) using text
+with ('text.delimiter'='||', 'text.null'='NULL') location ${table.path};
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table4_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table4_ddl.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table4_ddl.sql
new file mode 100644
index 0000000..2fb821a
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/multibytes_delimiter_table4_ddl.sql
@@ -0,0 +1,3 @@
+create external table table2 (id int, name text, score float, type text) using text
+with ('text.delimiter'='ㅎ', 'text.null'='NULL') location ${table.path};
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter3.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter3.sql
new file mode 100644
index 0000000..bd6b02d
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter3.sql
@@ -0,0 +1 @@
+select * from table1;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter4.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter4.sql
new file mode 100644
index 0000000..66a69ec
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testMultiBytesDelimiter4.sql
@@ -0,0 +1 @@
+select * from table2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter3.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter3.result b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter3.result
new file mode 100644
index 0000000..d8d43b1
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter3.result
@@ -0,0 +1,7 @@
+id,name,score,type
+-------------------------------
+1,ooo,1.1,a
+2,ppp,2.3,
+3,qqq,null,
+4,,4.5,
+null,xxx,5.6,e
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter4.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter4.result b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter4.result
new file mode 100644
index 0000000..d8d43b1
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestSelectQuery/testMultiBytesDelimiter4.result
@@ -0,0 +1,7 @@
+id,name,score,type
+-------------------------------
+1,ooo,1.1,a
+2,ppp,2.3,
+3,qqq,null,
+4,,4.5,
+null,xxx,5.6,e
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
index bb628b1..c1047d9 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -40,6 +40,7 @@ import org.apache.tajo.storage.compress.CodecPool;
 import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+import org.apache.tajo.util.Bytes;
 import org.apache.tajo.util.BytesUtils;
 
 import java.io.*;
@@ -83,7 +84,8 @@ public class CSVFile {
       this.meta = meta;
       this.schema = schema;
       this.delimiter = StringEscapeUtils.unescapeJava(
-          this.meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER)).getBytes();
+          this.meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER))
+          .getBytes(Bytes.UTF8_CHARSET);
 
       this.columnNum = schema.size();
 
@@ -93,7 +95,7 @@ public class CSVFile {
       if (StringUtils.isEmpty(nullCharacters)) {
         nullChars = NullDatum.get().asTextBytes();
       } else {
-        nullChars = nullCharacters.getBytes();
+        nullChars = nullCharacters.getBytes(Bytes.UTF8_CHARSET);
       }
     }
 
@@ -265,7 +267,8 @@ public class CSVFile {
       //Delimiter
       this.delimiter = StringEscapeUtils.unescapeJava(
           meta.getOption(StorageConstants.TEXT_DELIMITER,
-          meta.getOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER))).getBytes();
+          meta.getOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER)))
+          .getBytes(Bytes.UTF8_CHARSET);
 
       String nullCharacters = StringEscapeUtils.unescapeJava(
           meta.getOption(StorageConstants.TEXT_NULL,
@@ -274,7 +277,7 @@ public class CSVFile {
       if (StringUtils.isEmpty(nullCharacters)) {
         nullChars = NullDatum.get().asTextBytes();
       } else {
-        nullChars = nullCharacters.getBytes();
+        nullChars = nullCharacters.getBytes(Bytes.UTF8_CHARSET);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
index 1599f62..6a8c7a9 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.storage.text;
 
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufProcessor;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.datum.Datum;
@@ -28,9 +29,10 @@ import org.apache.tajo.storage.Tuple;
 import java.io.IOException;
 
 public class CSVLineDeserializer extends TextLineDeserializer {
-  private FieldSplitProcessor processor;
+  private ByteBufProcessor processor;
   private FieldSerializerDeserializer fieldSerDer;
   private ByteBuf nullChars;
+  private int delimiterCompensation;
 
   public CSVLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
     super(schema, meta, targetColumnIndexes);
@@ -38,7 +40,13 @@ public class CSVLineDeserializer extends TextLineDeserializer {
 
   @Override
   public void init() {
-    this.processor = new FieldSplitProcessor(CSVLineSerDe.getFieldDelimiter(meta));
+    byte[] delimiter = CSVLineSerDe.getFieldDelimiter(meta);
+    if (delimiter.length == 1) {
+      this.processor = new FieldSplitProcessor(delimiter[0]);
+    } else {
+      this.processor = new MultiBytesFieldSplitProcessor(delimiter);
+    }
+    this.delimiterCompensation = delimiter.length - 1;
 
     if (nullChars != null) {
       nullChars.release();
@@ -67,7 +75,7 @@ public class CSVLineDeserializer extends TextLineDeserializer {
       if (end < 0) {
         fieldLength = rowLength - start;
       } else {
-        fieldLength = end - start;
+        fieldLength = end - start - delimiterCompensation;
       }
 
       if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
index 2fe7f23..988d5d1 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
@@ -22,6 +22,7 @@ import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.Bytes;
 
 public class CSVLineSerDe extends TextLineSerDe {
   @Override
@@ -34,8 +35,8 @@ public class CSVLineSerDe extends TextLineSerDe {
     return new CSVLineSerializer(schema, meta);
   }
 
-  public static char getFieldDelimiter(TableMeta meta) {
+  public static byte[] getFieldDelimiter(TableMeta meta) {
     return StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_DELIMITER,
-        StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
+        StorageConstants.DEFAULT_FIELD_DELIMITER)).getBytes(Bytes.UTF8_CHARSET);
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
index 53a0ef3..9a2fe37 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
@@ -30,8 +30,8 @@ import java.io.OutputStream;
 public class CSVLineSerializer extends TextLineSerializer {
   private FieldSerializerDeserializer serde;
 
-  private byte [] nullChars;
-  private char delimiter;
+  private byte[] nullChars;
+  private byte[] delimiter;
   private int columnNum;
 
   public CSVLineSerializer(Schema schema, TableMeta meta) {
@@ -56,8 +56,8 @@ public class CSVLineSerializer extends TextLineSerializer {
       writtenBytes += serde.serialize(out, datum, schema.getColumn(i), i, nullChars);
 
       if (columnNum - 1 > i) {
-        out.write((byte) delimiter);
-        writtenBytes += 1;
+        out.write(delimiter);
+        writtenBytes += delimiter.length;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index ebf9608..4c9234e 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -391,7 +391,7 @@ public class DelimitedTextFile {
 
           try {
             deserializer.deserialize(buf, tuple);
-            // if a line is read normaly, it exists this loop.
+            // if a line is read normally, it exists this loop.
             break;
 
           } catch (TextLineParsingError tae) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
index a5ac142..862b5ae 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
@@ -21,9 +21,9 @@ package org.apache.tajo.storage.text;
 import io.netty.buffer.ByteBufProcessor;
 
 public class FieldSplitProcessor implements ByteBufProcessor {
-  private char delimiter; //the ascii separate character
+  private byte delimiter; //the ascii separate character
 
-  public FieldSplitProcessor(char recordDelimiterByte) {
+  public FieldSplitProcessor(byte recordDelimiterByte) {
     this.delimiter = recordDelimiterByte;
   }
 
@@ -31,8 +31,4 @@ public class FieldSplitProcessor implements ByteBufProcessor {
   public boolean process(byte value) throws Exception {
     return delimiter != value;
   }
-
-  public char getDelimiter() {
-    return delimiter;
-  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/MultiBytesFieldSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/MultiBytesFieldSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/MultiBytesFieldSplitProcessor.java
new file mode 100644
index 0000000..b97d7c6
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/MultiBytesFieldSplitProcessor.java
@@ -0,0 +1,45 @@
+/**
+ * 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.text;
+
+import io.netty.buffer.ByteBufProcessor;
+
+public class MultiBytesFieldSplitProcessor implements ByteBufProcessor {
+
+  private int index;
+  private final byte[] delimiter;
+
+  public MultiBytesFieldSplitProcessor(byte[] recordDelimiterByte) {
+    this.delimiter = recordDelimiterByte;
+  }
+
+  @Override
+  public boolean process(byte value) throws Exception {
+    if (delimiter[index] != value) {
+      index = 0;
+      return true;
+    }
+    if (index != delimiter.length - 1) {
+      index++;
+      return true;
+    }
+    index = 0;
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
index 7ebfa79..89a7de9 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
@@ -29,9 +29,9 @@ import java.io.IOException;
  * Reads a text line and fills a Tuple with values
  */
 public abstract class TextLineDeserializer {
-  protected Schema schema;
-  protected TableMeta meta;
-  protected int [] targetColumnIndexes;
+  protected final Schema schema;
+  protected final TableMeta meta;
+  protected final int[] targetColumnIndexes;
 
   public TextLineDeserializer(Schema schema, TableMeta meta, int [] targetColumnIndexes) {
     this.schema = schema;

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
index e81e289..1a53bb0 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
@@ -26,6 +26,7 @@ import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.BufferPool;
 import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.Bytes;
 
 /**
  * Pluggable Text Line SerDe class
@@ -56,7 +57,7 @@ public abstract class TextLineSerDe {
     if (StringUtils.isEmpty(nullCharacters)) {
       nullChars = NullDatum.get().asTextBytes();
     } else {
-      nullChars = nullCharacters.getBytes();
+      nullChars = nullCharacters.getBytes(Bytes.UTF8_CHARSET);
     }
 
     return nullChars;

http://git-wip-us.apache.org/repos/asf/tajo/blob/82d44af3/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
index 12ea551..2174d62 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
@@ -19,10 +19,12 @@
 package org.apache.tajo.storage;
 
 import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufProcessor;
 import io.netty.buffer.Unpooled;
 import io.netty.util.CharsetUtil;
 import org.apache.tajo.storage.text.FieldSplitProcessor;
 import org.apache.tajo.storage.text.LineSplitProcessor;
+import org.apache.tajo.storage.text.MultiBytesFieldSplitProcessor;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -35,17 +37,47 @@ public class TestSplitProcessor {
 
   @Test
   public void testFieldSplitProcessor() throws IOException {
-    String data = "abc||de";
+    String data = "abc||de|";
     final ByteBuf buf = releaseLater(
         Unpooled.copiedBuffer(data, CharsetUtil.ISO_8859_1));
 
     final int len = buf.readableBytes();
-    FieldSplitProcessor processor = new FieldSplitProcessor('|');
+    FieldSplitProcessor processor = new FieldSplitProcessor((byte)'|');
 
     assertEquals(3, buf.forEachByte(0, len, processor));
     assertEquals(4, buf.forEachByte(4, len - 4, processor));
-    assertEquals(-1, buf.forEachByte(5, len - 5, processor));
+    assertEquals(7, buf.forEachByte(5, len - 5, processor));
+    assertEquals(-1, buf.forEachByte(8, len - 8, processor));
+  }
+
+  @Test
+  public void testMultiCharFieldSplitProcessor1() throws IOException {
+    String data = "abc||||de||";
+    final ByteBuf buf = releaseLater(
+        Unpooled.copiedBuffer(data, CharsetUtil.ISO_8859_1));
+
+    final int len = buf.readableBytes();
+    ByteBufProcessor processor = new MultiBytesFieldSplitProcessor("||".getBytes());
+
+    assertEquals(4, buf.forEachByte(0, len, processor));
+    assertEquals(6, buf.forEachByte(5, len - 5, processor));
+    assertEquals(10, buf.forEachByte(7, len - 7, processor));
+    assertEquals(-1, buf.forEachByte(11, len - 11, processor));
+  }
+
+  @Test
+  public void testMultiCharFieldSplitProcessor2() throws IOException {
+    String data = "abcㅎㅎdeㅎ";
+    final ByteBuf buf = releaseLater(
+        Unpooled.copiedBuffer(data, CharsetUtil.UTF_8));
+
+    final int len = buf.readableBytes();
+    ByteBufProcessor processor = new MultiBytesFieldSplitProcessor("ㅎ".getBytes());
 
+    assertEquals(5, buf.forEachByte(0, len, processor));
+    assertEquals(8, buf.forEachByte(6, len - 6, processor));
+    assertEquals(13, buf.forEachByte(9, len - 9, processor));
+    assertEquals(-1, buf.forEachByte(14, len - 14, processor));
   }
 
   @Test


[13/13] tajo git commit: Merge branch 'index_support' of https://git-wip-us.apache.org/repos/asf/tajo into index_support

Posted by ji...@apache.org.
Merge branch 'index_support' of https://git-wip-us.apache.org/repos/asf/tajo into index_support


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

Branch: refs/heads/index_support
Commit: 60cbe9cac53873efe1d6cf352b5e93aa859d7bd7
Parents: 6b30cfe 63c8e1c
Author: Jihoon Son <ji...@apache.org>
Authored: Thu Mar 19 02:25:31 2015 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Thu Mar 19 02:25:31 2015 +0900

----------------------------------------------------------------------
 .../org/apache/tajo/algebra/CreateIndex.java    | 21 ++++++++++-
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  4 +-
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  5 ++-
 .../apache/tajo/master/exec/DDLExecutor.java    | 39 +++++++++++++++++++-
 .../apache/tajo/master/exec/QueryExecutor.java  |  5 +--
 .../tajo/engine/query/TestCreateIndex.java      | 15 ++++++++
 .../testCreateIndexOnLocation.sql               |  1 +
 .../org/apache/tajo/plan/LogicalPlanner.java    |  9 ++++-
 .../tajo/plan/logical/CreateIndexNode.java      | 17 +++++++--
 .../rules/LogicalPlanEqualityTester.java        |  1 -
 .../plan/serder/LogicalNodeDeserializer.java    |  1 +
 .../tajo/plan/serder/LogicalNodeSerializer.java |  1 +
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  2 +-
 tajo-plan/src/main/proto/Plan.proto             |  1 +
 14 files changed, 105 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/60cbe9ca/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/60cbe9ca/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/60cbe9ca/tajo-plan/src/main/proto/Plan.proto
----------------------------------------------------------------------


[10/13] tajo git commit: TAJO-1337: Implements common modules to handle RESTful API

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java
deleted file mode 100644
index ed6b634..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java
+++ /dev/null
@@ -1,182 +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.rpc;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public final class RpcChannelFactory {
-  private static final Log LOG = LogFactory.getLog(RpcChannelFactory.class);
-  
-  private static final int DEFAULT_WORKER_NUM = Runtime.getRuntime().availableProcessors() * 2;
-
-  private static final Object lockObjectForLoopGroup = new Object();
-  private static AtomicInteger serverCount = new AtomicInteger(0);
-
-  public enum ClientChannelId {
-    CLIENT_DEFAULT,
-    FETCHER
-  }
-
-  private static final Map<ClientChannelId, Integer> defaultMaxKeyPoolCount =
-      new ConcurrentHashMap<ClientChannelId, Integer>();
-  private static final Map<ClientChannelId, Queue<EventLoopGroup>> eventLoopGroupPool =
-      new ConcurrentHashMap<ClientChannelId, Queue<EventLoopGroup>>();
-
-  private RpcChannelFactory(){
-  }
-  
-  static {
-    Runtime.getRuntime().addShutdownHook(new CleanUpHandler());
-
-    defaultMaxKeyPoolCount.put(ClientChannelId.CLIENT_DEFAULT, 1);
-    defaultMaxKeyPoolCount.put(ClientChannelId.FETCHER, 1);
-  }
-
-  /**
-  * make this factory static thus all clients can share its thread pool.
-  * NioClientSocketChannelFactory has only one method newChannel() visible for user, which is thread-safe
-  */
-  public static EventLoopGroup getSharedClientEventloopGroup() {
-    return getSharedClientEventloopGroup(DEFAULT_WORKER_NUM);
-  }
-  
-  /**
-  * make this factory static thus all clients can share its thread pool.
-  * NioClientSocketChannelFactory has only one method newChannel() visible for user, which is thread-safe
-  *
-  * @param workerNum The number of workers
-  */
-  public static EventLoopGroup getSharedClientEventloopGroup(int workerNum){
-    //shared woker and boss pool
-    return getSharedClientEventloopGroup(ClientChannelId.CLIENT_DEFAULT, workerNum);
-  }
-
-  /**
-   * This function return eventloopgroup by key. Fetcher client will have one or more eventloopgroup for its throughput.
-   *
-   * @param clientId
-   * @param workerNum
-   * @return
-   */
-  public static EventLoopGroup getSharedClientEventloopGroup(ClientChannelId clientId, int workerNum) {
-    Queue<EventLoopGroup> eventLoopGroupQueue;
-    EventLoopGroup returnEventLoopGroup;
-
-    synchronized (lockObjectForLoopGroup) {
-      eventLoopGroupQueue = eventLoopGroupPool.get(clientId);
-      if (eventLoopGroupQueue == null) {
-        eventLoopGroupQueue = createClientEventloopGroups(clientId, workerNum);
-      }
-
-      returnEventLoopGroup = eventLoopGroupQueue.poll();
-      if (isEventLoopGroupShuttingDown(returnEventLoopGroup)) {
-        returnEventLoopGroup = createClientEventloopGroup(clientId.name(), workerNum);
-      }
-      eventLoopGroupQueue.add(returnEventLoopGroup);
-    }
-
-    return returnEventLoopGroup;
-  }
-
-  protected static boolean isEventLoopGroupShuttingDown(EventLoopGroup eventLoopGroup) {
-    return ((eventLoopGroup == null) || eventLoopGroup.isShuttingDown());
-  }
-
-  // Client must release the external resources
-  protected static Queue<EventLoopGroup> createClientEventloopGroups(ClientChannelId clientId, int workerNum) {
-    int defaultMaxObjectCount = defaultMaxKeyPoolCount.get(clientId);
-    Queue<EventLoopGroup> loopGroupQueue = new ConcurrentLinkedQueue<EventLoopGroup>();
-    eventLoopGroupPool.put(clientId, loopGroupQueue);
-
-    for (int objectIdx = 0; objectIdx < defaultMaxObjectCount; objectIdx++) {
-      loopGroupQueue.add(createClientEventloopGroup(clientId.name(), workerNum));
-    }
-
-    return loopGroupQueue;
-  }
-
-  protected static EventLoopGroup createClientEventloopGroup(String name, int workerNum) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Create " + name + " ClientEventLoopGroup. Worker:" + workerNum);
-    }
-
-    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
-    ThreadFactory clientFactory = builder.setNameFormat(name + " Client #%d").build();
-
-    return new NioEventLoopGroup(workerNum, clientFactory);
-  }
-
-  // Client must release the external resources
-  public static ServerBootstrap createServerChannelFactory(String name, int workerNum) {
-    name = name + "-" + serverCount.incrementAndGet();
-    if(LOG.isInfoEnabled()){
-      LOG.info("Create " + name + " ServerSocketChannelFactory. Worker:" + workerNum);
-    }
-    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
-    ThreadFactory bossFactory = builder.setNameFormat(name + " Server Boss #%d").build();
-    ThreadFactory workerFactory = builder.setNameFormat(name + " Server Worker #%d").build();
-    
-    EventLoopGroup bossGroup =
-        new NioEventLoopGroup(1, bossFactory);
-    EventLoopGroup workerGroup = 
-        new NioEventLoopGroup(workerNum, workerFactory);
-    
-    return new ServerBootstrap().group(bossGroup, workerGroup);
-  }
-
-  public static void shutdownGracefully(){
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Shutdown Shared RPC Pool");
-    }
-
-    synchronized(lockObjectForLoopGroup) {
-      for (Queue<EventLoopGroup> eventLoopGroupQueue: eventLoopGroupPool.values()) {
-        for (EventLoopGroup eventLoopGroup: eventLoopGroupQueue) {
-          eventLoopGroup.shutdownGracefully();
-        }
-
-        eventLoopGroupQueue.clear();
-      }
-      eventLoopGroupPool.clear();
-    }
-  }
-  
-  static class CleanUpHandler extends Thread {
-
-    @Override
-    public void run() {
-      RpcChannelFactory.shutdownGracefully();
-    }
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
deleted file mode 100644
index 6d1f479..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
+++ /dev/null
@@ -1,190 +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.rpc;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import io.netty.channel.ConnectTimeoutException;
-import io.netty.util.internal.logging.CommonsLoggerFactory;
-import io.netty.util.internal.logging.InternalLoggerFactory;
-
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-
-public class RpcConnectionPool {
-  private static final Log LOG = LogFactory.getLog(RpcConnectionPool.class);
-
-  private Map<RpcConnectionKey, NettyClientBase> connections =
-      new HashMap<RpcConnectionKey, NettyClientBase>();
-
-  private static RpcConnectionPool instance;
-  private final Object lockObject = new Object();
-
-  public final static int RPC_RETRIES = 3;
-
-  private RpcConnectionPool() {
-  }
-
-  public synchronized static RpcConnectionPool getPool() {
-    if(instance == null) {
-      InternalLoggerFactory.setDefaultFactory(new CommonsLoggerFactory());
-      instance = new RpcConnectionPool();
-    }
-    return instance;
-  }
-
-  private NettyClientBase makeConnection(RpcConnectionKey rpcConnectionKey)
-      throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
-    NettyClientBase client;
-    if(rpcConnectionKey.asyncMode) {
-      client = new AsyncRpcClient(rpcConnectionKey, RPC_RETRIES);
-    } else {
-      client = new BlockingRpcClient(rpcConnectionKey, RPC_RETRIES);
-    }
-    return client;
-  }
-
-  public static final long DEFAULT_TIMEOUT = 3000;
-  public static final long DEFAULT_INTERVAL = 500;
-
-  public NettyClientBase getConnection(InetSocketAddress addr,
-                                       Class<?> protocolClass, boolean asyncMode)
-      throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
-    return getConnection(addr, protocolClass, asyncMode, DEFAULT_TIMEOUT, DEFAULT_INTERVAL);
-  }
-
-  public NettyClientBase getConnection(InetSocketAddress addr,
-      Class<?> protocolClass, boolean asyncMode, long timeout, long interval)
-      throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
-    RpcConnectionKey key = new RpcConnectionKey(addr, protocolClass, asyncMode);
-
-    RpcUtils.Timer timer = new RpcUtils.Timer(timeout);
-    for (; !timer.isTimedOut(); timer.elapsed()) {
-      NettyClientBase client;
-      synchronized (lockObject) {
-        client = connections.get(key);
-        if (client == null) {
-          connections.put(key, client = makeConnection(key));
-        }
-      }
-      if (client.acquire(timer.remaining())) {
-        return client;
-      }
-      timer.interval(interval);
-    }
-
-    throw new ConnectTimeoutException("Failed to get connection for " + timeout + " msec");
-  }
-
-  public void releaseConnection(NettyClientBase client) {
-    release(client, false);
-  }
-
-  public void closeConnection(NettyClientBase client) {
-    release(client, true);
-  }
-
-  private void release(NettyClientBase client, boolean close) {
-    if (client == null) {
-      return;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Close connection [" + client.getKey() + "]");
-    }
-    try {
-      if (returnToPool(client, close)) {
-        client.close();
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Current Connections [" + connections.size() + "]");
-      }
-    } catch (Exception e) {
-      LOG.error("Can't close connection:" + client.getKey() + ":" + e.getMessage(), e);
-    }
-  }
-
-  // return true if the connection should be closed
-  private boolean returnToPool(NettyClientBase client, boolean close) {
-    synchronized (lockObject) {
-      if (client.release() && (close || !client.isConnected())) {
-        connections.remove(client.getKey());
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public void close() {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Pool Closed");
-    }
-
-    synchronized (lockObject) {
-      for (NettyClientBase eachClient : connections.values()) {
-        try {
-          eachClient.close();
-        } catch (Exception e) {
-          LOG.error("close client pool error", e);
-        }
-      }
-      connections.clear();
-    }
-  }
-
-  public void shutdown(){
-    close();
-    RpcChannelFactory.shutdownGracefully();
-  }
-
-  static class RpcConnectionKey {
-    final InetSocketAddress addr;
-    final Class<?> protocolClass;
-    final boolean asyncMode;
-
-    final String description;
-
-    public RpcConnectionKey(InetSocketAddress addr,
-                            Class<?> protocolClass, boolean asyncMode) {
-      this.addr = addr;
-      this.protocolClass = protocolClass;
-      this.asyncMode = asyncMode;
-      this.description = "["+ protocolClass + "] " + addr + "," + asyncMode;
-    }
-
-    @Override
-    public String toString() {
-      return description;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if(!(obj instanceof RpcConnectionKey)) {
-        return false;
-      }
-
-      return toString().equals(obj.toString());
-    }
-
-    @Override
-    public int hashCode() {
-      return description.hashCode();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java
deleted file mode 100644
index 152d426..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java
+++ /dev/null
@@ -1,122 +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.rpc;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.util.concurrent.atomic.AtomicReference;
-
-public class RpcUtils {
-
-  public static String normalizeInetSocketAddress(InetSocketAddress addr) {
-    return addr.getAddress().getHostAddress() + ":" + addr.getPort();
-  }
-
-  /**
-   * Util method to build socket addr from either:
-   *   <host>
-   *   <host>:<port>
-   *   <fs>://<host>:<port>/<path>
-   */
-  public static InetSocketAddress createSocketAddr(String host, int port) {
-    return new InetSocketAddress(host, port);
-  }
-
-  /**
-   * Returns InetSocketAddress that a client can use to
-   * connect to the server. NettyServerBase.getListenerAddress() is not correct when
-   * the server binds to "0.0.0.0". This returns "hostname:port" of the server,
-   * or "127.0.0.1:port" when the getListenerAddress() returns "0.0.0.0:port".
-   *
-   * @param addr of a listener
-   * @return socket address that a client can use to connect to the server.
-   */
-  public static InetSocketAddress getConnectAddress(InetSocketAddress addr) {
-    if (!addr.isUnresolved() && addr.getAddress().isAnyLocalAddress()) {
-      try {
-        addr = new InetSocketAddress(InetAddress.getLocalHost(), addr.getPort());
-      } catch (UnknownHostException uhe) {
-        // shouldn't get here unless the host doesn't have a loopback iface
-        addr = new InetSocketAddress("127.0.0.1", addr.getPort());
-      }
-    }
-    InetSocketAddress canonicalAddress =
-        new InetSocketAddress(addr.getAddress().getCanonicalHostName(), addr.getPort());
-    return canonicalAddress;
-  }
-
-  public static InetSocketAddress createUnresolved(String addr) {
-    String [] splitted = addr.split(":");
-    return InetSocketAddress.createUnresolved(splitted[0], Integer.parseInt(splitted[1]));
-  }
-
-  public static class Timer {
-    private long remaining;
-    private long prev;
-    public Timer(long timeout) {
-      this.remaining = timeout;
-      this.prev = System.currentTimeMillis();
-    }
-
-    public boolean isTimedOut() {
-      return remaining <= 0;
-    }
-
-    public void elapsed() {
-      long current = System.currentTimeMillis();
-      remaining -= (prev - current);
-      prev = current;
-    }
-
-    public void interval(long wait) {
-      if (wait <= 0 || isTimedOut()) {
-        return;
-      }
-      try {
-        Thread.sleep(Math.min(remaining, wait));
-      } catch (Exception ex) {
-        // ignore
-      }
-    }
-
-    public long remaining() {
-      return remaining;
-    }
-  }
-
-  public static class Scrutineer<T> {
-
-    private final AtomicReference<T> reference = new AtomicReference<T>();
-
-    T check(T ticket) {
-      T granted = reference.get();
-      for (;granted == null; granted = reference.get()) {
-        if (reference.compareAndSet(null, ticket)) {
-          return ticket;
-        }
-      }
-      return granted;
-    }
-
-    boolean clear(T granted) {
-      return reference.compareAndSet(granted, null);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/ServerCallable.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ServerCallable.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ServerCallable.java
deleted file mode 100644
index fb1cec2..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ServerCallable.java
+++ /dev/null
@@ -1,162 +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.rpc;
-
-import java.io.IOException;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.google.protobuf.ServiceException;
-
-public abstract class ServerCallable<T> {
-  protected InetSocketAddress addr;
-  protected long startTime;
-  protected long endTime;
-  protected Class<?> protocol;
-  protected boolean asyncMode;
-  protected boolean closeConn;
-  protected RpcConnectionPool connPool;
-
-  public abstract T call(NettyClientBase client) throws Exception;
-
-  public ServerCallable(RpcConnectionPool connPool,  InetSocketAddress addr, Class<?> protocol, boolean asyncMode) {
-    this(connPool, addr, protocol, asyncMode, false);
-  }
-
-  public ServerCallable(RpcConnectionPool connPool, InetSocketAddress addr, Class<?> protocol,
-                        boolean asyncMode, boolean closeConn) {
-    this.connPool = connPool;
-    this.addr = addr;
-    this.protocol = protocol;
-    this.asyncMode = asyncMode;
-    this.closeConn = closeConn;
-  }
-
-  public void beforeCall() {
-    this.startTime = System.currentTimeMillis();
-  }
-
-  public long getStartTime(){
-    return startTime;
-  }
-
-  public void afterCall() {
-    this.endTime = System.currentTimeMillis();
-  }
-
-  public long getEndTime(){
-    return endTime;
-  }
-
-  boolean abort = false;
-  public void abort() {
-    abort = true;
-  }
-  /**
-   * Run this instance with retries, timed waits,
-   * and refinds of missing regions.
-   *
-   * @param <T> the type of the return value
-   * @return an object of type T
-   * @throws com.google.protobuf.ServiceException if a remote or network exception occurs
-   */
-  public T withRetries() throws ServiceException {
-    //TODO configurable
-    final long pause = 500; //ms
-    final int numRetries = 3;
-    List<Throwable> exceptions = new ArrayList<Throwable>();
-
-    for (int tries = 0; tries < numRetries; tries++) {
-      NettyClientBase client = null;
-      try {
-        beforeCall();
-        if(addr != null) {
-          client = connPool.getConnection(addr, protocol, asyncMode);
-        }
-        return call(client);
-      } catch (IOException ioe) {
-        exceptions.add(ioe);
-        if(abort) {
-          throw new ServiceException(ioe.getMessage(), ioe);
-        }
-        if (tries == numRetries - 1) {
-          throw new ServiceException("Giving up after tries=" + tries, ioe);
-        }
-      } catch (Throwable t) {
-        throw new ServiceException(t);
-      } finally {
-        afterCall();
-        if(closeConn) {
-          connPool.closeConnection(client);
-        } else {
-          connPool.releaseConnection(client);
-        }
-      }
-      try {
-        Thread.sleep(pause * (tries + 1));
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new ServiceException("Giving up after tries=" + tries, e);
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Run this instance against the server once.
-   * @param <T> the type of the return value
-   * @return an object of type T
-   * @throws java.io.IOException if a remote or network exception occurs
-   * @throws RuntimeException other unspecified error
-   */
-  public T withoutRetries() throws IOException, RuntimeException {
-    NettyClientBase client = null;
-    try {
-      beforeCall();
-      client = connPool.getConnection(addr, protocol, asyncMode);
-      return call(client);
-    } catch (Throwable t) {
-      Throwable t2 = translateException(t);
-      if (t2 instanceof IOException) {
-        throw (IOException)t2;
-      } else {
-        throw new RuntimeException(t2);
-      }
-    } finally {
-      afterCall();
-      if(closeConn) {
-        connPool.closeConnection(client);
-      } else {
-        connPool.releaseConnection(client);
-      }
-    }
-  }
-
-  private static Throwable translateException(Throwable t) throws IOException {
-    if (t instanceof UndeclaredThrowableException) {
-      t = t.getCause();
-    }
-    if (t instanceof RemoteException && t.getCause() != null) {
-      t = t.getCause();
-    }
-    return t;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/TajoServiceException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/TajoServiceException.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/TajoServiceException.java
deleted file mode 100644
index 113d181..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/TajoServiceException.java
+++ /dev/null
@@ -1,58 +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.rpc;
-
-import com.google.protobuf.ServiceException;
-import org.apache.commons.lang.exception.ExceptionUtils;
-
-public class TajoServiceException extends ServiceException {
-  private String traceMessage;
-  private String protocol;
-  private String remoteAddress;
-
-  public TajoServiceException(String message) {
-    super(message);
-  }
-  public TajoServiceException(String message, String traceMessage) {
-    super(message);
-    this.traceMessage = traceMessage;
-  }
-
-  public TajoServiceException(String message, Throwable cause, String protocol, String remoteAddress) {
-    super(message, cause);
-
-    this.protocol = protocol;
-    this.remoteAddress = remoteAddress;
-  }
-
-  public String getTraceMessage() {
-    if(traceMessage == null && getCause() != null){
-      this.traceMessage = ExceptionUtils.getStackTrace(getCause());
-    }
-    return traceMessage;
-  }
-
-  public String getProtocol() {
-    return protocol;
-  }
-
-  public String getRemoteAddress() {
-    return remoteAddress;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/proto/DummyProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/proto/DummyProtos.proto b/tajo-rpc/src/main/proto/DummyProtos.proto
deleted file mode 100644
index f53f0d6..0000000
--- a/tajo-rpc/src/main/proto/DummyProtos.proto
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2012 Database Lab., Korea Univ.
- *
- * Licensed 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.
- */
-
-option java_package = "org.apache.tajo.rpc.test";
-option java_outer_classname = "DummyProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-message MulRequest1 {
-	required int32 x1 = 1;
-	required int32 x2 = 2;
-}
-
-message MulRequest2 {
-	required int32 x1 = 1;
-	required int32 x2 = 2;
-}
-
-message MulResponse {
-	required int32 result1 = 1;
-	required int32 result2 = 2;
-}
-
-message InnerNode {
-	required string instr = 1;
-}
-
-message InnerRequest {
-	repeated InnerNode nodes = 1;
-}
-
-message InnerResponse {
-	repeated InnerNode nodes = 1;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/proto/RpcProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/proto/RpcProtos.proto b/tajo-rpc/src/main/proto/RpcProtos.proto
deleted file mode 100644
index 69f43ed..0000000
--- a/tajo-rpc/src/main/proto/RpcProtos.proto
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2012 Database Lab., Korea Univ.
- *
- * Licensed 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.
- */
-
-option java_package = "org.apache.tajo.rpc";
-option java_outer_classname = "RpcProtos";
-
-message RpcRequest {
-  required int32 id = 1;
-  required string method_name = 2;
-  optional bytes request_message = 3;
-}
-
-message RpcResponse {
-  required int32 id = 1;
-  optional bytes response_message = 2;
-  optional string error_class = 3;
-  optional string error_message = 4;
-  optional string error_trace = 5;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/proto/TestProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/proto/TestProtocol.proto b/tajo-rpc/src/main/proto/TestProtocol.proto
deleted file mode 100644
index 58640ea..0000000
--- a/tajo-rpc/src/main/proto/TestProtocol.proto
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2012 Database Lab., Korea Univ.
- *
- * Licensed 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.
- */
-
-option java_package = "org.apache.tajo.rpc.test";
-option java_outer_classname = "DummyProtocol";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "TestProtos.proto";
-
-service DummyProtocolService {
-  rpc sum (SumRequest) returns (SumResponse);
-  rpc echo (EchoMessage) returns (EchoMessage);
-  rpc getError (EchoMessage) returns (EchoMessage);
-  rpc getNull (EchoMessage) returns (EchoMessage);
-  rpc deley (EchoMessage) returns (EchoMessage);
-  rpc throwException (EchoMessage) returns (EchoMessage);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/proto/TestProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/proto/TestProtos.proto b/tajo-rpc/src/main/proto/TestProtos.proto
deleted file mode 100644
index 5001c0e..0000000
--- a/tajo-rpc/src/main/proto/TestProtos.proto
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Copyright 2012 Database Lab., Korea Univ.
- *
- * Licensed 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.
- */
-
-option java_package = "org.apache.tajo.rpc.test";
-option java_outer_classname = "TestProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-message EchoMessage {
-  required string message = 1;
-}
-
-message SumRequest {
-  required int32 x1 = 1;
-  required int64 x2 = 2;
-  required double x3 = 3;
-  required float x4 = 4;
-}
-
-message SumResponse {
-  required double result = 1;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/test/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/log4j.properties b/tajo-rpc/src/test/java/log4j.properties
deleted file mode 100644
index 2c4d991..0000000
--- a/tajo-rpc/src/test/java/log4j.properties
+++ /dev/null
@@ -1,25 +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.
-#
-
-# log4j configuration used during build and unit tests
-
-log4j.rootLogger=info,stdout
-log4j.threshhold=ALL
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %p: %c (%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
deleted file mode 100644
index a974a65..0000000
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
+++ /dev/null
@@ -1,345 +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.rpc;
-
-import com.google.protobuf.RpcCallback;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.test.DummyProtocol;
-import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.Interface;
-import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
-import org.apache.tajo.rpc.test.TestProtos.SumRequest;
-import org.apache.tajo.rpc.test.TestProtos.SumResponse;
-import org.apache.tajo.rpc.test.impl.DummyProtocolAsyncImpl;
-import org.junit.AfterClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExternalResource;
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.*;
-
-public class TestAsyncRpc {
-  private static Log LOG = LogFactory.getLog(TestAsyncRpc.class);
-  private static String MESSAGE = "TestAsyncRpc";
-
-  double sum;
-  String echo;
-
-  AsyncRpcServer server;
-  AsyncRpcClient client;
-  Interface stub;
-  DummyProtocolAsyncImpl service;
-  int retries;
-  
-  @Retention(RetentionPolicy.RUNTIME)
-  @Target(ElementType.METHOD)
-  @interface SetupRpcConnection {
-    boolean setupRpcServer() default true;
-    boolean setupRpcClient() default true;
-  }
-  
-  @Rule
-  public ExternalResource resource = new ExternalResource() {
-    
-    private Description description;
-
-    @Override
-    public Statement apply(Statement base, Description description) {
-      this.description = description;
-      return super.apply(base, description);
-    }
-
-    @Override
-    protected void before() throws Throwable {
-      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
-
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
-        setUpRpcServer();
-      }
-      
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
-        setUpRpcClient();
-      }
-    }
-
-    @Override
-    protected void after() {
-      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
-
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
-        try {
-          tearDownRpcClient();
-        } catch (Exception e) {
-          fail(e.getMessage());
-        }
-      }
-      
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
-        try {
-          tearDownRpcServer();
-        } catch (Exception e) {
-          fail(e.getMessage());
-        }
-      }
-    }
-    
-  };
-  
-  public void setUpRpcServer() throws Exception {
-    service = new DummyProtocolAsyncImpl();
-    server = new AsyncRpcServer(DummyProtocol.class,
-        service, new InetSocketAddress("127.0.0.1", 0), 2);
-    server.start();
-  }
-  
-  public void setUpRpcClient() throws Exception {
-    retries = 1;
-
-    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(
-              RpcUtils.getConnectAddress(server.getListenAddress()),
-              DummyProtocol.class, true);
-    client = new AsyncRpcClient(rpcConnectionKey, retries);
-    client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT);
-    stub = client.getStub();
-  }
-
-  @AfterClass
-  public static void tearDownClass() throws Exception {
-    RpcChannelFactory.shutdownGracefully();
-  }
-  
-  public void tearDownRpcServer() throws Exception {
-    if(server != null) {
-      server.shutdown();
-      server = null;
-    }
-  }
-  
-  public void tearDownRpcClient() throws Exception {
-    if(client != null) {
-      client.close();
-      client = null;
-    }
-  }
-
-  boolean calledMarker = false;
-
-  @Test
-  public void testRpc() throws Exception {
-
-    SumRequest sumRequest = SumRequest.newBuilder()
-        .setX1(1)
-        .setX2(2)
-        .setX3(3.15d)
-        .setX4(2.0f).build();
-
-    stub.sum(null, sumRequest, new RpcCallback<SumResponse>() {
-      @Override
-      public void run(SumResponse parameter) {
-        sum = parameter.getResult();
-        assertTrue(8.15d == sum);
-      }
-    });
-
-
-    EchoMessage echoMessage = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    RpcCallback<EchoMessage> callback = new RpcCallback<EchoMessage>() {
-      @Override
-      public void run(EchoMessage parameter) {
-        echo = parameter.getMessage();
-        assertEquals(MESSAGE, echo);
-        calledMarker = true;
-      }
-    };
-    stub.echo(null, echoMessage, callback);
-    Thread.sleep(1000);
-    assertTrue(calledMarker);
-  }
-
-  private CountDownLatch testNullLatch;
-
-  @Test
-  public void testGetNull() throws Exception {
-    testNullLatch = new CountDownLatch(1);
-    stub.getNull(null, null, new RpcCallback<EchoMessage>() {
-      @Override
-      public void run(EchoMessage parameter) {
-        assertNull(parameter);
-        LOG.info("testGetNull retrieved");
-        testNullLatch.countDown();
-      }
-    });
-    assertTrue(testNullLatch.await(1000, TimeUnit.MILLISECONDS));
-    assertTrue(service.getNullCalled);
-  }
-
-  @Test
-  public void testCallFuture() throws Exception {
-    EchoMessage echoMessage = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
-    stub.deley(null, echoMessage, future);
-
-    assertFalse(future.isDone());
-    assertEquals(future.get(), echoMessage);
-    assertTrue(future.isDone());
-  }
-
-  @Test
-  public void testCallFutureTimeout() throws Exception {
-    boolean timeout = false;
-    try {
-      EchoMessage echoMessage = EchoMessage.newBuilder()
-          .setMessage(MESSAGE).build();
-      CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
-      stub.deley(null, echoMessage, future);
-
-      assertFalse(future.isDone());
-      future.get(1, TimeUnit.SECONDS);
-    } catch (TimeoutException te) {
-      timeout = true;
-    }
-    assertTrue(timeout);
-  }
-
-  @Test
-  public void testCallFutureDisconnected() throws Exception {
-    EchoMessage echoMessage = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
-
-    tearDownRpcServer();
-
-    stub.echo(future.getController(), echoMessage, future);
-    EchoMessage response = future.get();
-
-    assertNull(response);
-    assertTrue(future.getController().failed());
-    assertTrue(future.getController().errorText() != null);
-  }
-
-  @Test
-  public void testStubDisconnected() throws Exception {
-
-    EchoMessage echoMessage = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
-
-    if (server != null) {
-      server.shutdown(true);
-      server = null;
-    }
-
-    stub = client.getStub();
-    stub.echo(future.getController(), echoMessage, future);
-    EchoMessage response = future.get();
-
-    assertNull(response);
-    assertTrue(future.getController().failed());
-    assertTrue(future.getController().errorText() != null);
-  }
-
-  @Test
-  @SetupRpcConnection(setupRpcServer=false,setupRpcClient=false)
-  public void testConnectionRetry() throws Exception {
-    retries = 10;
-    ServerSocket serverSocket = new ServerSocket(0);
-    final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort());
-    serverSocket.close();
-    service = new DummyProtocolAsyncImpl();
-
-    EchoMessage echoMessage = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
-
-    //lazy startup
-    Thread serverThread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          Thread.sleep(1000);
-          server = new AsyncRpcServer(DummyProtocol.class,
-              service, address, 2);
-        } catch (Exception e) {
-          fail(e.getMessage());
-        }
-        server.start();
-      }
-    });
-    serverThread.start();
-
-    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, true);
-    client = new AsyncRpcClient(rpcConnectionKey, retries);
-    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
-    stub = client.getStub();
-    stub.echo(future.getController(), echoMessage, future);
-
-    assertFalse(future.isDone());
-    assertEquals(echoMessage, future.get());
-    assertTrue(future.isDone());
-  }
-
-  @Test
-  public void testConnectionFailure() throws Exception {
-    InetSocketAddress address = new InetSocketAddress("test", 0);
-    try {
-      RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, true);
-      NettyClientBase client = new AsyncRpcClient(rpcConnectionKey, retries);
-      assertFalse(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
-    } catch (Throwable throwable) {
-      fail();
-    }
-  }
-
-  @Test
-  @SetupRpcConnection(setupRpcClient=false)
-  public void testUnresolvedAddress() throws Exception {
-    String hostAndPort = RpcUtils.normalizeInetSocketAddress(server.getListenAddress());
-    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(
-              RpcUtils.createUnresolved(hostAndPort), DummyProtocol.class, true);
-    client = new AsyncRpcClient(rpcConnectionKey, retries);
-    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
-    Interface stub = client.getStub();
-    EchoMessage echoMessage = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    CallFuture<EchoMessage> future = new CallFuture<EchoMessage>();
-    stub.deley(null, echoMessage, future);
-
-    assertFalse(future.isDone());
-    assertEquals(future.get(), echoMessage);
-    assertTrue(future.isDone());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
deleted file mode 100644
index 10dd766..0000000
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
+++ /dev/null
@@ -1,349 +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.rpc;
-
-import org.apache.tajo.rpc.test.DummyProtocol;
-import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.BlockingInterface;
-import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
-import org.apache.tajo.rpc.test.TestProtos.SumRequest;
-import org.apache.tajo.rpc.test.TestProtos.SumResponse;
-import org.apache.tajo.rpc.test.impl.DummyProtocolBlockingImpl;
-import org.junit.AfterClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExternalResource;
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
-
-public class TestBlockingRpc {
-  public static final String MESSAGE = "TestBlockingRpc";
-
-  private BlockingRpcServer server;
-  private BlockingRpcClient client;
-  private BlockingInterface stub;
-  private DummyProtocolBlockingImpl service;
-  private int retries;
-  
-  @Retention(RetentionPolicy.RUNTIME)
-  @Target(ElementType.METHOD)
-  @interface SetupRpcConnection {
-    boolean setupRpcServer() default true;
-    boolean setupRpcClient() default true;
-  }
-  
-  @Rule
-  public ExternalResource resource = new ExternalResource() {
-    
-    private Description description;
-
-    @Override
-    public Statement apply(Statement base, Description description) {
-      this.description = description;
-      return super.apply(base, description);
-    }
-
-    @Override
-    protected void before() throws Throwable {
-      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
-      
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
-        setUpRpcServer();
-      }
-      
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
-        setUpRpcClient();
-      }
-    }
-
-    @Override
-    protected void after() {
-      SetupRpcConnection setupRpcConnection = description.getAnnotation(SetupRpcConnection.class);
-      
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcClient()) {
-        try {
-          tearDownRpcClient();
-        } catch (Exception e) {
-          fail(e.getMessage());
-        }
-      }
-      
-      if (setupRpcConnection == null || setupRpcConnection.setupRpcServer()) {
-        try {
-          tearDownRpcServer();
-        } catch (Exception e) {
-          fail(e.getMessage());
-        }
-      }
-    }
-    
-  };
-  
-  public void setUpRpcServer() throws Exception {
-    service = new DummyProtocolBlockingImpl();
-    server = new BlockingRpcServer(DummyProtocol.class, service,
-        new InetSocketAddress("127.0.0.1", 0), 2);
-    server.start();
-  }
-  
-  public void setUpRpcClient() throws Exception {
-    retries = 1;
-
-    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(
-              RpcUtils.getConnectAddress(server.getListenAddress()),
-              DummyProtocol.class, false);
-    client = new BlockingRpcClient(rpcConnectionKey, retries);
-    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
-    stub = client.getStub();
-  }
-
-  @AfterClass
-  public static void tearDownClass() throws Exception {
-    RpcChannelFactory.shutdownGracefully();
-  }
-  
-  public void tearDownRpcServer() throws Exception {
-    if(server != null) {
-      server.shutdown();
-      server = null;
-    }
-  }
-  
-  public void tearDownRpcClient() throws Exception {
-    if(client != null) {
-      client.close();
-      client = null;
-    }
-  }
-
-  @Test
-  public void testRpc() throws Exception {
-    SumRequest request = SumRequest.newBuilder()
-        .setX1(1)
-        .setX2(2)
-        .setX3(3.15d)
-        .setX4(2.0f).build();
-    SumResponse response1 = stub.sum(null, request);
-    assertEquals(8.15d, response1.getResult(), 1e-15);
-
-    EchoMessage message = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    EchoMessage response2 = stub.echo(null, message);
-    assertEquals(MESSAGE, response2.getMessage());
-  }
-
-  @Test
-  @SetupRpcConnection(setupRpcClient=false)
-  public void testRpcWithServiceCallable() throws Exception {
-    RpcConnectionPool pool = RpcConnectionPool.getPool();
-    final SumRequest request = SumRequest.newBuilder()
-        .setX1(1)
-        .setX2(2)
-        .setX3(3.15d)
-        .setX4(2.0f).build();
-
-    SumResponse response =
-    new ServerCallable<SumResponse>(pool,
-        server.getListenAddress(), DummyProtocol.class, false) {
-      @Override
-      public SumResponse call(NettyClientBase client) throws Exception {
-        BlockingInterface stub2 = client.getStub();
-        SumResponse response1 = stub2.sum(null, request);
-        return response1;
-      }
-    }.withRetries();
-
-    assertEquals(8.15d, response.getResult(), 1e-15);
-
-    response =
-        new ServerCallable<SumResponse>(pool,
-            server.getListenAddress(), DummyProtocol.class, false) {
-          @Override
-          public SumResponse call(NettyClientBase client) throws Exception {
-            BlockingInterface stub2 = client.getStub();
-            SumResponse response1 = stub2.sum(null, request);
-            return response1;
-          }
-        }.withoutRetries();
-
-    assertTrue(8.15d == response.getResult());
-    pool.close();
-  }
-
-  @Test
-  public void testThrowException() throws Exception {
-    EchoMessage message = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-
-    try {
-      stub.throwException(null, message);
-      fail("RpcCall should throw exception");
-    } catch (Throwable t) {
-      assertTrue(t instanceof TajoServiceException);
-      assertEquals("Exception Test", t.getMessage());
-      TajoServiceException te = (TajoServiceException)t;
-      assertEquals("org.apache.tajo.rpc.test.DummyProtocol", te.getProtocol());
-      assertEquals(server.getListenAddress().getAddress().getHostAddress() + ":" + server.getListenAddress().getPort(),
-          te.getRemoteAddress());
-    }
-  }
-
-  @Test
-  @SetupRpcConnection(setupRpcServer=false,setupRpcClient=false)
-  public void testConnectionRetry() throws Exception {
-    retries = 10;
-    ServerSocket serverSocket = new ServerSocket(0);
-    final InetSocketAddress address = new InetSocketAddress("127.0.0.1", serverSocket.getLocalPort());
-    serverSocket.close();
-
-    EchoMessage message = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-
-    //lazy startup
-    Thread serverThread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          Thread.sleep(1000);
-          server = new BlockingRpcServer(DummyProtocol.class, new DummyProtocolBlockingImpl(), address, 2);
-        } catch (Exception e) {
-          fail(e.getMessage());
-        }
-        server.start();
-      }
-    });
-    serverThread.start();
-
-    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, false);
-    client = new BlockingRpcClient(rpcConnectionKey, retries);
-    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
-    stub = client.getStub();
-
-    EchoMessage response = stub.echo(null, message);
-    assertEquals(MESSAGE, response.getMessage());
-  }
-
-  @Test
-  public void testConnectionFailed() throws Exception {
-    NettyClientBase client = null;
-    
-    try {
-      int port = server.getListenAddress().getPort() + 1;
-      RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(
-              RpcUtils.getConnectAddress(new InetSocketAddress("127.0.0.1", port)),
-              DummyProtocol.class, false);
-      client = new BlockingRpcClient(rpcConnectionKey, retries);
-      assertFalse(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
-      client.close();
-    } catch (Throwable ce){
-      if (client != null) {
-        client.close();
-      }
-      fail();
-    }
-  }
-
-  @Test
-  public void testGetNull() throws Exception {
-    assertNull(stub.getNull(null, null));
-    assertTrue(service.getNullCalled);
-  }
-
-  @Test
-  public void testShutdown() throws Exception {
-    final StringBuilder error = new StringBuilder();
-    Thread callThread = new Thread() {
-      public void run() {
-        try {
-          EchoMessage message = EchoMessage.newBuilder()
-              .setMessage(MESSAGE)
-              .build();
-          stub.deley(null, message);
-        } catch (Exception e) {
-          error.append(e.getMessage());
-        }
-        synchronized(error) {
-          error.notifyAll();
-        }
-      }
-    };
-
-    callThread.start();
-
-    final CountDownLatch latch = new CountDownLatch(1);
-    Thread shutdownThread = new Thread() {
-      public void run() {
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-        }
-        try {
-          server.shutdown();
-          server = null;
-          latch.countDown();
-        } catch (Throwable e) {
-          e.printStackTrace();
-        }
-      }
-    };
-    shutdownThread.start();
-
-    assertTrue(latch.await(5 * 1000, TimeUnit.MILLISECONDS));
-
-    assertTrue(latch.getCount() == 0);
-
-    synchronized(error) {
-      error.wait(5 * 1000);
-    }
-
-    if(!error.toString().isEmpty()) {
-      fail(error.toString());
-    }
-  }
-
-  @Test
-  @SetupRpcConnection(setupRpcClient=false)
-  public void testUnresolvedAddress() throws Exception {
-    String hostAndPort = RpcUtils.normalizeInetSocketAddress(server.getListenAddress());
-    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
-          new RpcConnectionPool.RpcConnectionKey(
-              RpcUtils.createUnresolved(hostAndPort), DummyProtocol.class, false);
-    client = new BlockingRpcClient(rpcConnectionKey, retries);
-    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
-    BlockingInterface stub = client.getStub();
-
-    EchoMessage message = EchoMessage.newBuilder()
-        .setMessage(MESSAGE).build();
-    EchoMessage response2 = stub.echo(null, message);
-    assertEquals(MESSAGE, response2.getMessage());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java
deleted file mode 100644
index 0ca7563..0000000
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolAsyncImpl.java
+++ /dev/null
@@ -1,86 +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.rpc.test.impl;
-
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.Interface;
-import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
-import org.apache.tajo.rpc.test.TestProtos.SumRequest;
-import org.apache.tajo.rpc.test.TestProtos.SumResponse;
-
-public class DummyProtocolAsyncImpl implements Interface {
-  private static final Log LOG =
-      LogFactory.getLog(DummyProtocolAsyncImpl.class);
-  public boolean getNullCalled = false;
-  public boolean getErrorCalled = false;
-
-  @Override
-  public void sum(RpcController controller, SumRequest request,
-                  RpcCallback<SumResponse> done) {
-
-    SumResponse response = SumResponse.newBuilder().setResult(
-        request.getX1()+request.getX2()+request.getX3()+request.getX4()
-    ).build();
-    done.run(response);
-  }
-
-  @Override
-  public void echo(RpcController controller, EchoMessage request,
-                   RpcCallback<EchoMessage> done) {
-
-    done.run(request);
-  }
-
-  @Override
-  public void getError(RpcController controller, EchoMessage request,
-                       RpcCallback<EchoMessage> done) {
-    LOG.info("noCallback is called");
-    getErrorCalled = true;
-    controller.setFailed(request.getMessage());
-    done.run(request);
-  }
-
-  @Override
-  public void getNull(RpcController controller, EchoMessage request,
-                      RpcCallback<EchoMessage> done) {
-    getNullCalled = true;
-    LOG.info("noCallback is called");
-    done.run(null);
-  }
-
-  @Override
-  public void deley(RpcController controller, EchoMessage request,
-                    RpcCallback<EchoMessage> done) {
-    try {
-      Thread.sleep(3000);
-    } catch (InterruptedException e) {
-      LOG.error(e.getMessage());
-    }
-
-    done.run(request);
-  }
-
-  public void throwException(RpcController controller, EchoMessage request,
-                             RpcCallback<EchoMessage> done) {
-    done.run(request);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
deleted file mode 100644
index 8d4b597..0000000
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
+++ /dev/null
@@ -1,83 +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.rpc.test.impl;
-
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.test.DummyProtocol.DummyProtocolService.BlockingInterface;
-import org.apache.tajo.rpc.test.TestProtos.EchoMessage;
-import org.apache.tajo.rpc.test.TestProtos.SumRequest;
-import org.apache.tajo.rpc.test.TestProtos.SumResponse;
-
-public class DummyProtocolBlockingImpl implements BlockingInterface {
-  private static final Log LOG =
-      LogFactory.getLog(DummyProtocolBlockingImpl.class);
-  public boolean getNullCalled = false;
-  public boolean getErrorCalled = false;
-
-  @Override
-  public SumResponse sum(RpcController controller, SumRequest request)
-      throws ServiceException {
-    return SumResponse.newBuilder().setResult(
-        request.getX1()+request.getX2()+request.getX3()+request.getX4()
-    ).build();
-  }
-
-  @Override
-  public EchoMessage echo(RpcController controller, EchoMessage request)
-      throws ServiceException {
-    return EchoMessage.newBuilder().
-        setMessage(request.getMessage()).build();
-  }
-
-  @Override
-  public EchoMessage getError(RpcController controller, EchoMessage request)
-      throws ServiceException {
-    getErrorCalled = true;
-    controller.setFailed(request.getMessage());
-    return request;
-  }
-
-  @Override
-  public EchoMessage getNull(RpcController controller, EchoMessage request)
-      throws ServiceException {
-    getNullCalled = true;
-    LOG.info("noCallback is called");
-    return null;
-  }
-
-  @Override
-  public EchoMessage deley(RpcController controller, EchoMessage request)
-      throws ServiceException {
-    try {
-      Thread.sleep(3000);
-    } catch (InterruptedException e) {
-      //throw new ServiceException(e.getMessage(), e);
-    }
-
-    return request;
-  }
-
-  public EchoMessage throwException(RpcController controller, EchoMessage request)
-      throws ServiceException {
-    throw new ServiceException("Exception Test");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-common/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-common/pom.xml b/tajo-rpc/tajo-rpc-common/pom.xml
new file mode 100644
index 0000000..2b1cd7a
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-common/pom.xml
@@ -0,0 +1,216 @@
+<!--
+  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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <version>0.11.0-SNAPSHOT</version>
+    <groupId>org.apache.tajo</groupId>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <packaging>jar</packaging>
+  <artifactId>tajo-rpc-common</artifactId>
+  <name>Tajo Rpc Common</name>
+  <description>Common Implementation for Netty</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <configuration>
+        </configuration>
+        <executions>
+          <execution>
+            <id>create-jar</id>
+            <phase>prepare-package</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-transport</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-handler</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+  </properties>
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+			</url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                    <echo file="${project.build.directory}/dist-layout-stitching.sh">
+                      run() {
+                      echo "\$ ${@}"
+                      "${@}"
+                      res=$?
+                      if [ $res != 0 ]; then
+                      echo
+                      echo "Failed!"
+                      echo
+                      exit $res
+                      fi
+                      }
+
+                      ROOT=`cd ${basedir}/..;pwd`
+                      echo
+                      echo "Current directory `pwd`"
+                      echo
+                      run rm -rf ${project.artifactId}-${project.version}
+                      run mkdir ${project.artifactId}-${project.version}
+                      run cd ${project.artifactId}-${project.version}
+                      run cp -r ${basedir}/target/${project.artifactId}-${project.version}*.jar .
+                      echo
+                      echo "Tajo Rpc dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo
+                    </echo>
+                    <exec executable="sh" dir="${project.build.directory}" failonerror="true">
+                      <arg line="./dist-layout-stitching.sh" />
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+
+</project>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/NettyServerBase.java b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
new file mode 100644
index 0000000..ad443d7
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
@@ -0,0 +1,243 @@
+/**
+ * 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.rpc;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.util.concurrent.GlobalEventExecutor;
+
+import java.io.IOException;
+import java.net.DatagramSocket;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Base class for netty implementation.
+ */
+public class NettyServerBase {
+  private static final Log LOG = LogFactory.getLog(NettyServerBase.class);
+  private static final String DEFAULT_PREFIX = "RpcServer_";
+  private static final AtomicInteger sequenceId = new AtomicInteger(0);
+
+  protected String serviceName;
+  protected InetSocketAddress serverAddr;
+  protected InetSocketAddress bindAddress;
+  protected ChannelInitializer<Channel> initializer;
+  protected ServerBootstrap bootstrap;
+  protected ChannelFuture channelFuture;
+  protected ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
+
+  private InetSocketAddress initIsa;
+  private Set<RpcEventListener> listeners = Collections.synchronizedSet(new HashSet<RpcEventListener>());
+
+  public NettyServerBase(InetSocketAddress address) {
+    this.initIsa = address;
+  }
+
+  public NettyServerBase(String serviceName, InetSocketAddress addr) {
+    this.serviceName = serviceName;
+    this.initIsa = addr;
+  }
+
+  public void setName(String name) {
+    this.serviceName = name;
+  }
+
+  public void init(ChannelInitializer<Channel> initializer, int workerNum) {
+    for (RpcEventListener listener: listeners) {
+      listener.onBeforeInit(this);
+    }
+    
+    bootstrap = RpcChannelFactory.createServerChannelFactory(serviceName, workerNum);
+
+    this.initializer = initializer;
+    bootstrap
+      .channel(NioServerSocketChannel.class)
+      .childHandler(initializer)
+      .option(ChannelOption.SO_REUSEADDR, true)
+      .option(ChannelOption.TCP_NODELAY, true)
+      .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
+      .childOption(ChannelOption.TCP_NODELAY, true)
+      .childOption(ChannelOption.CONNECT_TIMEOUT_MILLIS, 10000)
+      .childOption(ChannelOption.SO_RCVBUF, 1048576 * 10);
+    
+    for (RpcEventListener listener: listeners) {
+      listener.onAfterInit(this);
+    }
+  }
+
+  public InetSocketAddress getListenAddress() {
+    return this.bindAddress;
+  }
+
+  public void start() {
+    for (RpcEventListener listener: listeners) {
+      listener.onBeforeStart(this);
+    }
+    
+    if (serviceName == null) {
+      this.serviceName = getNextDefaultServiceName();
+    }
+
+    if (initIsa.getPort() == 0) {
+      try {
+        int port = getUnusedPort();
+        serverAddr = new InetSocketAddress(initIsa.getHostName(), port);
+      } catch (IOException e) {
+        LOG.error(e, e);
+      }
+    } else {
+      serverAddr = initIsa;
+    }
+
+    this.channelFuture = bootstrap.clone().bind(serverAddr).syncUninterruptibly();
+    this.bindAddress = (InetSocketAddress) channelFuture.channel().localAddress();
+
+    for (RpcEventListener listener: listeners) {
+      listener.onAfterStart(this);
+    }
+    LOG.info("Rpc (" + serviceName + ") listens on " + this.bindAddress);
+  }
+
+  public Channel getChannel() {
+    return this.channelFuture.channel();
+  }
+
+  public void shutdown() {
+    shutdown(false);
+  }
+
+  public void shutdown(boolean waitUntilThreadsStop) {
+    for (RpcEventListener listener: listeners) {
+      listener.onBeforeShutdown(this);
+    }
+    
+    try {
+      accepted.close();
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+    }
+
+    if(bootstrap != null) {
+      if (bootstrap.childGroup() != null) {
+        bootstrap.childGroup().shutdownGracefully();
+        if (waitUntilThreadsStop) {
+          bootstrap.childGroup().terminationFuture().awaitUninterruptibly();
+        }
+      }
+
+      if (bootstrap.group() != null) {
+        bootstrap.group().shutdownGracefully();
+        if (waitUntilThreadsStop) {
+          bootstrap.childGroup().terminationFuture().awaitUninterruptibly();
+        }
+      }
+    }
+    
+    for (RpcEventListener listener: listeners) {
+      listener.onAfterShutdown(this);
+    }
+
+    if (bindAddress != null) {
+      LOG.info("Rpc (" + serviceName + ") listened on "
+          + RpcUtils.normalizeInetSocketAddress(bindAddress)+ ") shutdown");
+    }
+  }
+
+  private static String getNextDefaultServiceName() {
+    return DEFAULT_PREFIX + sequenceId.getAndIncrement();
+  }
+
+  private static final int startPortRange = 10000;
+  private static final int endPortRange = 50000;
+  private static final Random rnd = new Random(System.currentTimeMillis());
+  // each system has a different starting port number within the given range.
+  private static final AtomicInteger nextPortNum =
+      new AtomicInteger(startPortRange+ rnd.nextInt(endPortRange - startPortRange));
+  private static final Object lockObject = new Object();
+
+
+  private synchronized static int getUnusedPort() throws IOException {
+    while (true) {
+      int port = nextPortNum.getAndIncrement();
+      if (port >= endPortRange) {
+        synchronized (lockObject) {
+          nextPortNum.set(startPortRange);
+          port = nextPortNum.getAndIncrement();
+        }
+      }
+      if (available(port)) {
+        return port;
+      }
+    }
+  }
+
+  private static boolean available(int port) throws IOException {
+    if (port < 1024 || port > 65535) {
+      throw new IllegalArgumentException("Port Number Out of Bound: " + port);
+    }
+
+    ServerSocket ss = null;
+    DatagramSocket ds = null;
+
+    try {
+      ss = new ServerSocket(port);
+      ss.setReuseAddress(true);
+
+      ds = new DatagramSocket(port);
+      ds.setReuseAddress(true);
+
+      return true;
+
+    } catch (IOException e) {
+      return false;
+    } finally {
+      if (ss != null) {
+        ss.close();
+      }
+
+      if (ds != null) {
+        ds.close();
+      }
+    }
+  }
+  
+  public void addListener(RpcEventListener listener) {
+    listeners.add(listener);
+  }
+  
+  public void removeListener(RpcEventListener listener) {
+    listeners.remove(listener);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RemoteException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RemoteException.java b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RemoteException.java
new file mode 100644
index 0000000..30c110d
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RemoteException.java
@@ -0,0 +1,37 @@
+/**
+ * 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.rpc;
+
+public class RemoteException extends RuntimeException {
+  public RemoteException() {
+    super();
+  }
+
+  public RemoteException(String message) {
+    super(message);
+  }
+
+  public RemoteException(Throwable t) {
+    super(t);
+  }
+
+  public RemoteException(String message, Throwable t) {
+    super(message, t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
new file mode 100644
index 0000000..3c054ad
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
@@ -0,0 +1,104 @@
+/**
+ * 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.rpc;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+public class RetriesExhaustedException extends RuntimeException {
+  private static final long serialVersionUID = 1876775844L;
+
+  public RetriesExhaustedException(final String msg) {
+    super(msg);
+  }
+
+  public RetriesExhaustedException(final String msg, final IOException e) {
+    super(msg, e);
+  }
+
+  /**
+   * Datastructure that allows adding more info around Throwable incident.
+   */
+  public static class ThrowableWithExtraContext {
+    private final Throwable t;
+    private final long when;
+    private final String extras;
+
+    public ThrowableWithExtraContext(final Throwable t, final long when,
+        final String extras) {
+      this.t = t;
+      this.when = when;
+      this.extras = extras;
+    }
+ 
+    @Override
+    public String toString() {
+      return new Date(this.when).toString() + ", " + extras + ", " + t.toString();
+    }
+  }
+
+  /**
+   * Create a new RetriesExhaustedException from the list of prior failures.
+   * @param callableVitals Details from the {@link ServerCallable} we were using
+   * when we got this exception.
+   * @param numTries The number of tries we made
+   * @param exceptions List of exceptions that failed before giving up
+   */
+  public RetriesExhaustedException(final String callableVitals, int numTries,
+      List<Throwable> exceptions) {
+    super(getMessage(callableVitals, numTries, exceptions));
+  }
+
+  /**
+   * Create a new RetriesExhaustedException from the list of prior failures.
+   * @param numTries
+   * @param exceptions List of exceptions that failed before giving up
+   */
+  public RetriesExhaustedException(final int numTries,
+      final List<Throwable> exceptions) {
+    super(getMessage(numTries, exceptions));
+  }
+
+  private static String getMessage(String callableVitals, int numTries,
+      List<Throwable> exceptions) {
+    StringBuilder buffer = new StringBuilder("Failed contacting ");
+    buffer.append(callableVitals);
+    buffer.append(" after ");
+    buffer.append(numTries + 1);
+    buffer.append(" attempts.\nExceptions:\n");
+    for (Throwable t : exceptions) {
+      buffer.append(t.toString());
+      buffer.append("\n");
+    }
+    return buffer.toString();
+  }
+
+  private static String getMessage(final int numTries,
+      final List<Throwable> exceptions) {
+    StringBuilder buffer = new StringBuilder("Failed after attempts=");
+    buffer.append(numTries + 1);
+    buffer.append(", exceptions:\n");
+    for (Throwable t : exceptions) {
+      buffer.append(t.toString());
+      buffer.append("\n");
+    }
+    return buffer.toString();
+  }
+}
\ No newline at end of file


[09/13] tajo git commit: TAJO-1337: Implements common modules to handle RESTful API

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java
new file mode 100644
index 0000000..ed6b634
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcChannelFactory.java
@@ -0,0 +1,182 @@
+/**
+ * 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.rpc;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public final class RpcChannelFactory {
+  private static final Log LOG = LogFactory.getLog(RpcChannelFactory.class);
+  
+  private static final int DEFAULT_WORKER_NUM = Runtime.getRuntime().availableProcessors() * 2;
+
+  private static final Object lockObjectForLoopGroup = new Object();
+  private static AtomicInteger serverCount = new AtomicInteger(0);
+
+  public enum ClientChannelId {
+    CLIENT_DEFAULT,
+    FETCHER
+  }
+
+  private static final Map<ClientChannelId, Integer> defaultMaxKeyPoolCount =
+      new ConcurrentHashMap<ClientChannelId, Integer>();
+  private static final Map<ClientChannelId, Queue<EventLoopGroup>> eventLoopGroupPool =
+      new ConcurrentHashMap<ClientChannelId, Queue<EventLoopGroup>>();
+
+  private RpcChannelFactory(){
+  }
+  
+  static {
+    Runtime.getRuntime().addShutdownHook(new CleanUpHandler());
+
+    defaultMaxKeyPoolCount.put(ClientChannelId.CLIENT_DEFAULT, 1);
+    defaultMaxKeyPoolCount.put(ClientChannelId.FETCHER, 1);
+  }
+
+  /**
+  * make this factory static thus all clients can share its thread pool.
+  * NioClientSocketChannelFactory has only one method newChannel() visible for user, which is thread-safe
+  */
+  public static EventLoopGroup getSharedClientEventloopGroup() {
+    return getSharedClientEventloopGroup(DEFAULT_WORKER_NUM);
+  }
+  
+  /**
+  * make this factory static thus all clients can share its thread pool.
+  * NioClientSocketChannelFactory has only one method newChannel() visible for user, which is thread-safe
+  *
+  * @param workerNum The number of workers
+  */
+  public static EventLoopGroup getSharedClientEventloopGroup(int workerNum){
+    //shared woker and boss pool
+    return getSharedClientEventloopGroup(ClientChannelId.CLIENT_DEFAULT, workerNum);
+  }
+
+  /**
+   * This function return eventloopgroup by key. Fetcher client will have one or more eventloopgroup for its throughput.
+   *
+   * @param clientId
+   * @param workerNum
+   * @return
+   */
+  public static EventLoopGroup getSharedClientEventloopGroup(ClientChannelId clientId, int workerNum) {
+    Queue<EventLoopGroup> eventLoopGroupQueue;
+    EventLoopGroup returnEventLoopGroup;
+
+    synchronized (lockObjectForLoopGroup) {
+      eventLoopGroupQueue = eventLoopGroupPool.get(clientId);
+      if (eventLoopGroupQueue == null) {
+        eventLoopGroupQueue = createClientEventloopGroups(clientId, workerNum);
+      }
+
+      returnEventLoopGroup = eventLoopGroupQueue.poll();
+      if (isEventLoopGroupShuttingDown(returnEventLoopGroup)) {
+        returnEventLoopGroup = createClientEventloopGroup(clientId.name(), workerNum);
+      }
+      eventLoopGroupQueue.add(returnEventLoopGroup);
+    }
+
+    return returnEventLoopGroup;
+  }
+
+  protected static boolean isEventLoopGroupShuttingDown(EventLoopGroup eventLoopGroup) {
+    return ((eventLoopGroup == null) || eventLoopGroup.isShuttingDown());
+  }
+
+  // Client must release the external resources
+  protected static Queue<EventLoopGroup> createClientEventloopGroups(ClientChannelId clientId, int workerNum) {
+    int defaultMaxObjectCount = defaultMaxKeyPoolCount.get(clientId);
+    Queue<EventLoopGroup> loopGroupQueue = new ConcurrentLinkedQueue<EventLoopGroup>();
+    eventLoopGroupPool.put(clientId, loopGroupQueue);
+
+    for (int objectIdx = 0; objectIdx < defaultMaxObjectCount; objectIdx++) {
+      loopGroupQueue.add(createClientEventloopGroup(clientId.name(), workerNum));
+    }
+
+    return loopGroupQueue;
+  }
+
+  protected static EventLoopGroup createClientEventloopGroup(String name, int workerNum) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Create " + name + " ClientEventLoopGroup. Worker:" + workerNum);
+    }
+
+    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
+    ThreadFactory clientFactory = builder.setNameFormat(name + " Client #%d").build();
+
+    return new NioEventLoopGroup(workerNum, clientFactory);
+  }
+
+  // Client must release the external resources
+  public static ServerBootstrap createServerChannelFactory(String name, int workerNum) {
+    name = name + "-" + serverCount.incrementAndGet();
+    if(LOG.isInfoEnabled()){
+      LOG.info("Create " + name + " ServerSocketChannelFactory. Worker:" + workerNum);
+    }
+    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
+    ThreadFactory bossFactory = builder.setNameFormat(name + " Server Boss #%d").build();
+    ThreadFactory workerFactory = builder.setNameFormat(name + " Server Worker #%d").build();
+    
+    EventLoopGroup bossGroup =
+        new NioEventLoopGroup(1, bossFactory);
+    EventLoopGroup workerGroup = 
+        new NioEventLoopGroup(workerNum, workerFactory);
+    
+    return new ServerBootstrap().group(bossGroup, workerGroup);
+  }
+
+  public static void shutdownGracefully(){
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Shutdown Shared RPC Pool");
+    }
+
+    synchronized(lockObjectForLoopGroup) {
+      for (Queue<EventLoopGroup> eventLoopGroupQueue: eventLoopGroupPool.values()) {
+        for (EventLoopGroup eventLoopGroup: eventLoopGroupQueue) {
+          eventLoopGroup.shutdownGracefully();
+        }
+
+        eventLoopGroupQueue.clear();
+      }
+      eventLoopGroupPool.clear();
+    }
+  }
+  
+  static class CleanUpHandler extends Thread {
+
+    @Override
+    public void run() {
+      RpcChannelFactory.shutdownGracefully();
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcEventListener.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcEventListener.java b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcEventListener.java
new file mode 100644
index 0000000..4d72536
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcEventListener.java
@@ -0,0 +1,62 @@
+/**
+ * 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.rpc;
+
+/**
+ * Event listener for netty code. Users can subscribe events by using this interface.
+ */
+public interface RpcEventListener {
+
+  /**
+   * Performs actions before start.
+   * @param obj Method caller
+   */
+  public void onBeforeStart(Object obj);
+  
+  /**
+   * Performs actions after start.
+   * @param obj Method caller
+   */
+  public void onAfterStart(Object obj);
+  
+  /**
+   * Performs actions before initialization.
+   * @param obj Method caller
+   */
+  public void onBeforeInit(Object obj);
+  
+  /**
+   * Performs actions after initialization.
+   * @param obj Method caller
+   */
+  public void onAfterInit(Object obj);
+  
+  /**
+   * Performs actions before shutdown.
+   * @param obj Method caller
+   */
+  public void onBeforeShutdown(Object obj);
+  
+  /**
+   * Performs actions after shutdown.
+   * @param obj Method caller
+   */
+  public void onAfterShutdown(Object obj);
+  
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcUtils.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcUtils.java b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcUtils.java
new file mode 100644
index 0000000..152d426
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-common/src/main/java/org/apache/tajo/rpc/RpcUtils.java
@@ -0,0 +1,122 @@
+/*
+ * 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.rpc;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class RpcUtils {
+
+  public static String normalizeInetSocketAddress(InetSocketAddress addr) {
+    return addr.getAddress().getHostAddress() + ":" + addr.getPort();
+  }
+
+  /**
+   * Util method to build socket addr from either:
+   *   <host>
+   *   <host>:<port>
+   *   <fs>://<host>:<port>/<path>
+   */
+  public static InetSocketAddress createSocketAddr(String host, int port) {
+    return new InetSocketAddress(host, port);
+  }
+
+  /**
+   * Returns InetSocketAddress that a client can use to
+   * connect to the server. NettyServerBase.getListenerAddress() is not correct when
+   * the server binds to "0.0.0.0". This returns "hostname:port" of the server,
+   * or "127.0.0.1:port" when the getListenerAddress() returns "0.0.0.0:port".
+   *
+   * @param addr of a listener
+   * @return socket address that a client can use to connect to the server.
+   */
+  public static InetSocketAddress getConnectAddress(InetSocketAddress addr) {
+    if (!addr.isUnresolved() && addr.getAddress().isAnyLocalAddress()) {
+      try {
+        addr = new InetSocketAddress(InetAddress.getLocalHost(), addr.getPort());
+      } catch (UnknownHostException uhe) {
+        // shouldn't get here unless the host doesn't have a loopback iface
+        addr = new InetSocketAddress("127.0.0.1", addr.getPort());
+      }
+    }
+    InetSocketAddress canonicalAddress =
+        new InetSocketAddress(addr.getAddress().getCanonicalHostName(), addr.getPort());
+    return canonicalAddress;
+  }
+
+  public static InetSocketAddress createUnresolved(String addr) {
+    String [] splitted = addr.split(":");
+    return InetSocketAddress.createUnresolved(splitted[0], Integer.parseInt(splitted[1]));
+  }
+
+  public static class Timer {
+    private long remaining;
+    private long prev;
+    public Timer(long timeout) {
+      this.remaining = timeout;
+      this.prev = System.currentTimeMillis();
+    }
+
+    public boolean isTimedOut() {
+      return remaining <= 0;
+    }
+
+    public void elapsed() {
+      long current = System.currentTimeMillis();
+      remaining -= (prev - current);
+      prev = current;
+    }
+
+    public void interval(long wait) {
+      if (wait <= 0 || isTimedOut()) {
+        return;
+      }
+      try {
+        Thread.sleep(Math.min(remaining, wait));
+      } catch (Exception ex) {
+        // ignore
+      }
+    }
+
+    public long remaining() {
+      return remaining;
+    }
+  }
+
+  public static class Scrutineer<T> {
+
+    private final AtomicReference<T> reference = new AtomicReference<T>();
+
+    T check(T ticket) {
+      T granted = reference.get();
+      for (;granted == null; granted = reference.get()) {
+        if (reference.compareAndSet(null, ticket)) {
+          return ticket;
+        }
+      }
+      return granted;
+    }
+
+    boolean clear(T granted) {
+      return reference.compareAndSet(granted, null);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/pom.xml b/tajo-rpc/tajo-rpc-protobuf/pom.xml
new file mode 100644
index 0000000..1f67255
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/pom.xml
@@ -0,0 +1,274 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <version>0.11.0-SNAPSHOT</version>
+    <groupId>org.apache.tajo</groupId>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <packaging>jar</packaging>
+  <artifactId>tajo-rpc-protobuf</artifactId>
+  <name>Tajo Protocol Buffer Rpc</name>
+  <description>RPC Server/Client Implementation based on Netty and Protocol Buffer</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <configuration>
+        </configuration>
+        <executions>
+          <execution>
+            <id>create-jar</id>
+            <phase>prepare-package</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/DummyProtos.proto</argument>
+                <argument>src/main/proto/RpcProtos.proto</argument>
+                <argument>src/main/proto/TestProtos.proto</argument>
+                <argument>src/main/proto/TestProtocol.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-rpc-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+  </properties>
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+			</url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                    <echo file="${project.build.directory}/dist-layout-stitching.sh">
+                      run() {
+                      echo "\$ ${@}"
+                      "${@}"
+                      res=$?
+                      if [ $res != 0 ]; then
+                      echo
+                      echo "Failed!"
+                      echo
+                      exit $res
+                      fi
+                      }
+
+                      ROOT=`cd ${basedir}/..;pwd`
+                      echo
+                      echo "Current directory `pwd`"
+                      echo
+                      run rm -rf ${project.artifactId}-${project.version}
+                      run mkdir ${project.artifactId}-${project.version}
+                      run cd ${project.artifactId}-${project.version}
+                      run cp -r ${basedir}/target/${project.artifactId}-${project.version}*.jar .
+                      echo
+                      echo "Tajo Rpc dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo
+                    </echo>
+                    <exec executable="sh" dir="${project.build.directory}" failonerror="true">
+                      <arg line="./dist-layout-stitching.sh" />
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+
+</project>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
new file mode 100644
index 0000000..3d856ce
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
@@ -0,0 +1,227 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.*;
+
+import io.netty.channel.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
+import org.apache.tajo.rpc.RpcProtos.RpcRequest;
+import org.apache.tajo.rpc.RpcProtos.RpcResponse;
+
+import io.netty.util.ReferenceCountUtil;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class AsyncRpcClient extends NettyClientBase {
+  private static final Log LOG = LogFactory.getLog(AsyncRpcClient.class);
+
+  private final ConcurrentMap<Integer, ResponseCallback> requests =
+      new ConcurrentHashMap<Integer, ResponseCallback>();
+
+  private final Method stubMethod;
+  private final ProxyRpcChannel rpcChannel;
+  private final ClientChannelInboundHandler inboundHandler;
+
+  /**
+   * Intentionally make this method package-private, avoiding user directly
+   * new an instance through this constructor.
+   */
+  AsyncRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
+      throws ClassNotFoundException, NoSuchMethodException {
+    super(rpcConnectionKey, retries);
+    stubMethod = getServiceClass().getMethod("newStub", RpcChannel.class);
+    rpcChannel = new ProxyRpcChannel();
+    inboundHandler = new ClientChannelInboundHandler();
+    init(new ProtoChannelInitializer(inboundHandler, RpcResponse.getDefaultInstance()));
+  }
+
+  @Override
+  public <T> T getStub() {
+    return getStub(stubMethod, rpcChannel);
+  }
+
+  protected void sendExceptions(String message) {
+    for(Map.Entry<Integer, ResponseCallback> callbackEntry: requests.entrySet()) {
+      ResponseCallback callback = callbackEntry.getValue();
+      Integer id = callbackEntry.getKey();
+
+      RpcResponse.Builder responseBuilder = RpcResponse.newBuilder()
+          .setErrorMessage(message)
+          .setId(id);
+
+      callback.run(responseBuilder.build());
+    }
+  }
+
+  @Override
+  public void close() {
+    sendExceptions("AsyncRpcClient terminates all the connections");
+
+    super.close();
+  }
+
+  private class ProxyRpcChannel implements RpcChannel {
+
+    public void callMethod(final MethodDescriptor method,
+                           final RpcController controller,
+                           final Message param,
+                           final Message responseType,
+                           RpcCallback<Message> done) {
+
+      int nextSeqId = sequence.getAndIncrement();
+
+      Message rpcRequest = buildRequest(nextSeqId, method, param);
+
+      inboundHandler.registerCallback(nextSeqId,
+          new ResponseCallback(controller, responseType, done));
+
+      ChannelPromise channelPromise = getChannel().newPromise();
+      channelPromise.addListener(new GenericFutureListener<ChannelFuture>() {
+
+        @Override
+        public void operationComplete(ChannelFuture future) throws Exception {
+          if (!future.isSuccess()) {
+            inboundHandler.exceptionCaught(null, new ServiceException(future.cause()));
+          }
+        }
+      });
+      getChannel().writeAndFlush(rpcRequest, channelPromise);
+    }
+
+    private Message buildRequest(int seqId,
+                                 MethodDescriptor method,
+                                 Message param) {
+
+      RpcRequest.Builder requestBuilder = RpcRequest.newBuilder()
+          .setId(seqId)
+          .setMethodName(method.getName());
+
+      if (param != null) {
+          requestBuilder.setRequestMessage(param.toByteString());
+      }
+
+      return requestBuilder.build();
+    }
+  }
+
+  private class ResponseCallback implements RpcCallback<RpcResponse> {
+    private final RpcController controller;
+    private final Message responsePrototype;
+    private final RpcCallback<Message> callback;
+
+    public ResponseCallback(RpcController controller,
+                            Message responsePrototype,
+                            RpcCallback<Message> callback) {
+      this.controller = controller;
+      this.responsePrototype = responsePrototype;
+      this.callback = callback;
+    }
+
+    @Override
+    public void run(RpcResponse rpcResponse) {
+      // if hasErrorMessage is true, it means rpc-level errors.
+      // it does not call the callback function\
+      if (rpcResponse.hasErrorMessage()) {
+        if (controller != null) {
+          this.controller.setFailed(rpcResponse.getErrorMessage());
+        }
+        callback.run(null);
+      } else { // if rpc call succeed
+        try {
+          Message responseMessage;
+          if (!rpcResponse.hasResponseMessage()) {
+            responseMessage = null;
+          } else {
+            responseMessage = responsePrototype.newBuilderForType().mergeFrom(
+                rpcResponse.getResponseMessage()).build();
+          }
+
+          callback.run(responseMessage);
+
+        } catch (InvalidProtocolBufferException e) {
+          throw new RemoteException(getErrorMessage(""), e);
+        }
+      }
+    }
+  }
+
+  private String getErrorMessage(String message) {
+    return "Exception [" + protocol.getCanonicalName() +
+        "(" + RpcUtils.normalizeInetSocketAddress((InetSocketAddress)
+        getChannel().remoteAddress()) + ")]: " + message;
+  }
+
+  @ChannelHandler.Sharable
+  private class ClientChannelInboundHandler extends ChannelInboundHandlerAdapter {
+
+    void registerCallback(int seqId, ResponseCallback callback) {
+
+      if (requests.putIfAbsent(seqId, callback) != null) {
+        throw new RemoteException(
+            getErrorMessage("Duplicate Sequence Id "+ seqId));
+      }
+    }
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg)
+        throws Exception {
+      if (msg instanceof RpcResponse) {
+        try {
+          RpcResponse response = (RpcResponse) msg;
+          ResponseCallback callback = requests.remove(response.getId());
+
+          if (callback == null) {
+            LOG.warn("Dangling rpc call");
+          } else {
+            callback.run(response);
+          }
+        } finally {
+          ReferenceCountUtil.release(msg);
+        }
+      }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+        throws Exception {
+      LOG.error(getRemoteAddress() + "," + protocol + "," + cause.getMessage(), cause);
+
+      sendExceptions(cause.getMessage());
+      
+      if(LOG.isDebugEnabled()) {
+        LOG.error(cause.getMessage(), cause);
+      } else {
+        LOG.error("RPC Exception:" + cause.getMessage());
+      }
+      
+      if (ctx != null && ctx.channel().isActive()) {
+        ctx.channel().close();
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java
new file mode 100644
index 0000000..3b5a747
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java
@@ -0,0 +1,148 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.*;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+
+import io.netty.channel.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcProtos.RpcRequest;
+import org.apache.tajo.rpc.RpcProtos.RpcResponse;
+
+import io.netty.util.ReferenceCountUtil;
+
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+
+public class AsyncRpcServer extends NettyServerBase {
+  private static final Log LOG = LogFactory.getLog(AsyncRpcServer.class);
+
+  private final Service service;
+  private final ChannelInitializer<Channel> initializer;
+
+  public AsyncRpcServer(final Class<?> protocol,
+                        final Object instance,
+                        final InetSocketAddress bindAddress,
+                        final int workerNum)
+      throws Exception {
+    super(protocol.getSimpleName(), bindAddress);
+
+    String serviceClassName = protocol.getName() + "$" +
+        protocol.getSimpleName() + "Service";
+    Class<?> serviceClass = Class.forName(serviceClassName);
+    Class<?> interfaceClass = Class.forName(serviceClassName + "$Interface");
+    Method method = serviceClass.getMethod("newReflectiveService", interfaceClass);
+    this.service = (Service) method.invoke(null, instance);
+
+    this.initializer = new ProtoChannelInitializer(new ServerHandler(), RpcRequest.getDefaultInstance());
+    super.init(this.initializer, workerNum);
+  }
+
+  @ChannelHandler.Sharable
+  private class ServerHandler extends ChannelInboundHandlerAdapter {
+
+    @Override
+    public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
+      accepted.add(ctx.channel());
+      if(LOG.isDebugEnabled()){
+        LOG.debug(String.format(serviceName + " accepted number of connections (%d)", accepted.size()));
+      }
+      super.channelRegistered(ctx);
+    }
+
+    @Override
+    public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
+      accepted.remove(ctx.channel());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(serviceName + " closes a connection. The number of current connections are " + accepted.size());
+      }
+      super.channelUnregistered(ctx);
+    }
+
+    @Override
+    public void channelRead(final ChannelHandlerContext ctx, Object msg)
+        throws Exception {
+      if (msg instanceof RpcRequest) {
+        try {
+          final RpcRequest request = (RpcRequest) msg;
+
+          String methodName = request.getMethodName();
+          MethodDescriptor methodDescriptor = service.getDescriptorForType().findMethodByName(methodName);
+
+          if (methodDescriptor == null) {
+            throw new RemoteCallException(request.getId(), new NoSuchMethodException(methodName));
+          }
+
+          Message paramProto = null;
+          if (request.hasRequestMessage()) {
+            try {
+              paramProto = service.getRequestPrototype(methodDescriptor).newBuilderForType()
+                  .mergeFrom(request.getRequestMessage()).build();
+            } catch (Throwable t) {
+              throw new RemoteCallException(request.getId(), methodDescriptor, t);
+            }
+          }
+
+          final RpcController controller = new NettyRpcController();
+
+          RpcCallback<Message> callback = !request.hasId() ? null : new RpcCallback<Message>() {
+
+            public void run(Message returnValue) {
+
+              RpcResponse.Builder builder = RpcResponse.newBuilder().setId(request.getId());
+
+              if (returnValue != null) {
+                builder.setResponseMessage(returnValue.toByteString());
+              }
+
+              if (controller.failed()) {
+                builder.setErrorMessage(controller.errorText());
+              }
+
+              ctx.writeAndFlush(builder.build());
+            }
+          };
+
+          service.callMethod(methodDescriptor, controller, paramProto, callback);
+
+        } finally {
+          ReferenceCountUtil.release(msg);
+        }
+      }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+        throws Exception{
+      if (cause instanceof RemoteCallException) {
+        RemoteCallException callException = (RemoteCallException) cause;
+        ctx.writeAndFlush(callException.getResponse());
+      } else {
+        LOG.error(cause.getMessage());
+      }
+      
+      if (ctx != null && ctx.channel().isActive()) {
+        ctx.channel().close();
+      }
+    }
+    
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
new file mode 100644
index 0000000..6a90330
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
@@ -0,0 +1,273 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.*;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+
+import io.netty.channel.*;
+import io.netty.util.concurrent.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
+import org.apache.tajo.rpc.RpcProtos.RpcRequest;
+import org.apache.tajo.rpc.RpcProtos.RpcResponse;
+
+import io.netty.util.ReferenceCountUtil;
+
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.*;
+import java.util.concurrent.Future;
+
+public class BlockingRpcClient extends NettyClientBase {
+  private static final Log LOG = LogFactory.getLog(RpcProtos.class);
+
+  private final Map<Integer, ProtoCallFuture> requests =
+      new ConcurrentHashMap<Integer, ProtoCallFuture>();
+
+  private final Method stubMethod;
+  private final ProxyRpcChannel rpcChannel;
+  private final ChannelInboundHandlerAdapter inboundHandler;
+
+  /**
+   * Intentionally make this method package-private, avoiding user directly
+   * new an instance through this constructor.
+   */
+  BlockingRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
+      throws ClassNotFoundException, NoSuchMethodException {
+    super(rpcConnectionKey, retries);
+    stubMethod = getServiceClass().getMethod("newBlockingStub", BlockingRpcChannel.class);
+    rpcChannel = new ProxyRpcChannel();
+    inboundHandler = new ClientChannelInboundHandler();
+    init(new ProtoChannelInitializer(inboundHandler, RpcResponse.getDefaultInstance()));
+  }
+
+  @Override
+  public <T> T getStub() {
+    return getStub(stubMethod, rpcChannel);
+  }
+
+  @Override
+  public void close() {
+    for(ProtoCallFuture callback: requests.values()) {
+      callback.setFailed("BlockingRpcClient terminates all the connections",
+          new ServiceException("BlockingRpcClient terminates all the connections"));
+    }
+
+    super.close();
+  }
+
+  private class ProxyRpcChannel implements BlockingRpcChannel {
+
+    @Override
+    public Message callBlockingMethod(final MethodDescriptor method,
+                                      final RpcController controller,
+                                      final Message param,
+                                      final Message responsePrototype)
+        throws TajoServiceException {
+
+      int nextSeqId = sequence.getAndIncrement();
+
+      Message rpcRequest = buildRequest(nextSeqId, method, param);
+
+      ProtoCallFuture callFuture =
+          new ProtoCallFuture(controller, responsePrototype);
+      requests.put(nextSeqId, callFuture);
+
+      ChannelPromise channelPromise = getChannel().newPromise();
+      channelPromise.addListener(new GenericFutureListener<ChannelFuture>() {
+        @Override
+        public void operationComplete(ChannelFuture future) throws Exception {
+          if (!future.isSuccess()) {
+            inboundHandler.exceptionCaught(null, new ServiceException(future.cause()));
+          }
+        }
+      });
+      getChannel().writeAndFlush(rpcRequest, channelPromise);
+
+      try {
+        return callFuture.get(60, TimeUnit.SECONDS);
+      } catch (Throwable t) {
+        if (t instanceof ExecutionException) {
+          Throwable cause = t.getCause();
+          if (cause != null && cause instanceof TajoServiceException) {
+            throw (TajoServiceException)cause;
+          }
+        }
+        throw new TajoServiceException(t.getMessage());
+      }
+    }
+
+    private Message buildRequest(int seqId,
+                                 MethodDescriptor method,
+                                 Message param) {
+      RpcRequest.Builder requestBuilder = RpcRequest.newBuilder()
+          .setId(seqId)
+          .setMethodName(method.getName());
+
+      if (param != null) {
+        requestBuilder.setRequestMessage(param.toByteString());
+      }
+
+      return requestBuilder.build();
+    }
+  }
+
+  private String getErrorMessage(String message) {
+    if(getChannel() != null) {
+      return protocol.getName() +
+          "(" + RpcUtils.normalizeInetSocketAddress((InetSocketAddress)
+          getChannel().remoteAddress()) + "): " + message;
+    } else {
+      return "Exception " + message;
+    }
+  }
+
+  private TajoServiceException makeTajoServiceException(RpcResponse response, Throwable cause) {
+    if(getChannel() != null) {
+      return new TajoServiceException(response.getErrorMessage(), cause, protocol.getName(),
+          RpcUtils.normalizeInetSocketAddress((InetSocketAddress)getChannel().remoteAddress()));
+    } else {
+      return new TajoServiceException(response.getErrorMessage());
+    }
+  }
+
+  @ChannelHandler.Sharable
+  private class ClientChannelInboundHandler extends ChannelInboundHandlerAdapter {
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg)
+        throws Exception {
+
+      if (msg instanceof RpcResponse) {
+        try {
+          RpcResponse rpcResponse = (RpcResponse) msg;
+          ProtoCallFuture callback = requests.remove(rpcResponse.getId());
+
+          if (callback == null) {
+            LOG.warn("Dangling rpc call");
+          } else {
+            if (rpcResponse.hasErrorMessage()) {
+              callback.setFailed(rpcResponse.getErrorMessage(),
+                  makeTajoServiceException(rpcResponse, new ServiceException(rpcResponse.getErrorTrace())));
+              throw new RemoteException(getErrorMessage(rpcResponse.getErrorMessage()));
+            } else {
+              Message responseMessage;
+
+              if (!rpcResponse.hasResponseMessage()) {
+                responseMessage = null;
+              } else {
+                responseMessage = callback.returnType.newBuilderForType().mergeFrom(rpcResponse.getResponseMessage())
+                    .build();
+              }
+
+              callback.setResponse(responseMessage);
+            }
+          }
+        } finally {
+          ReferenceCountUtil.release(msg);
+        }
+      }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
+        throws Exception {
+      for(ProtoCallFuture callback: requests.values()) {
+        callback.setFailed(cause.getMessage(), cause);
+      }
+      
+      if(LOG.isDebugEnabled()) {
+        LOG.error("" + cause.getMessage(), cause);
+      } else {
+        LOG.error("RPC Exception:" + cause.getMessage());
+      }
+      if (ctx != null && ctx.channel().isActive()) {
+        ctx.channel().close();
+      }
+    }
+  }
+
+ static class ProtoCallFuture implements Future<Message> {
+    private Semaphore sem = new Semaphore(0);
+    private Message response = null;
+    private Message returnType;
+
+    private RpcController controller;
+
+    private ExecutionException ee;
+
+    public ProtoCallFuture(RpcController controller, Message message) {
+      this.controller = controller;
+      this.returnType = message;
+    }
+
+    @Override
+    public boolean cancel(boolean arg0) {
+      return false;
+    }
+
+    @Override
+    public Message get() throws InterruptedException, ExecutionException {
+      sem.acquire();
+      if(ee != null) {
+        throw ee;
+      }
+      return response;
+    }
+
+    @Override
+    public Message get(long timeout, TimeUnit unit)
+        throws InterruptedException, ExecutionException, TimeoutException {
+      if(sem.tryAcquire(timeout, unit)) {
+        if (ee != null) {
+          throw ee;
+        }
+        return response;
+      } else {
+        throw new TimeoutException();
+      }
+    }
+
+    @Override
+    public boolean isCancelled() {
+      return false;
+    }
+
+    @Override
+    public boolean isDone() {
+      return sem.availablePermits() > 0;
+    }
+
+    public void setResponse(Message response) {
+      this.response = response;
+      sem.release();
+    }
+
+    public void setFailed(String errorText, Throwable t) {
+      if(controller != null) {
+        this.controller.setFailed(errorText);
+      }
+      ee = new ExecutionException(errorText, t);
+      sem.release();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java
new file mode 100644
index 0000000..0ce359f
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java
@@ -0,0 +1,147 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcController;
+
+import io.netty.channel.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcProtos.RpcRequest;
+import org.apache.tajo.rpc.RpcProtos.RpcResponse;
+
+import io.netty.util.ReferenceCountUtil;
+
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+
+public class BlockingRpcServer extends NettyServerBase {
+  private static Log LOG = LogFactory.getLog(BlockingRpcServer.class);
+  private final BlockingService service;
+  private final ChannelInitializer<Channel> initializer;
+
+  public BlockingRpcServer(final Class<?> protocol,
+                           final Object instance,
+                           final InetSocketAddress bindAddress,
+                           final int workerNum)
+      throws Exception {
+
+    super(protocol.getSimpleName(), bindAddress);
+
+    String serviceClassName = protocol.getName() + "$" +
+        protocol.getSimpleName() + "Service";
+    Class<?> serviceClass = Class.forName(serviceClassName);
+    Class<?> interfaceClass = Class.forName(serviceClassName +
+        "$BlockingInterface");
+    Method method = serviceClass.getMethod(
+        "newReflectiveBlockingService", interfaceClass);
+
+    this.service = (BlockingService) method.invoke(null, instance);
+    this.initializer = new ProtoChannelInitializer(new ServerHandler(), RpcRequest.getDefaultInstance());
+
+    super.init(this.initializer, workerNum);
+  }
+
+  @ChannelHandler.Sharable
+  private class ServerHandler extends ChannelInboundHandlerAdapter {
+
+    @Override
+    public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
+      accepted.add(ctx.channel());
+      if(LOG.isDebugEnabled()){
+        LOG.debug(String.format(serviceName + " accepted number of connections (%d)", accepted.size()));
+      }
+      super.channelRegistered(ctx);
+    }
+
+    @Override
+    public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
+      accepted.remove(ctx.channel());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(serviceName + " closes a connection. The number of current connections are " + accepted.size());
+      }
+      super.channelUnregistered(ctx);
+    }
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg)
+        throws Exception {
+
+      if (msg instanceof RpcRequest) {
+        try {
+          final RpcRequest request = (RpcRequest) msg;
+
+          String methodName = request.getMethodName();
+          MethodDescriptor methodDescriptor = service.getDescriptorForType().findMethodByName(methodName);
+
+          if (methodDescriptor == null) {
+            throw new RemoteCallException(request.getId(), new NoSuchMethodException(methodName));
+          }
+          Message paramProto = null;
+          if (request.hasRequestMessage()) {
+            try {
+              paramProto = service.getRequestPrototype(methodDescriptor).newBuilderForType()
+                  .mergeFrom(request.getRequestMessage()).build();
+
+            } catch (Throwable t) {
+              throw new RemoteCallException(request.getId(), methodDescriptor, t);
+            }
+          }
+          Message returnValue;
+          RpcController controller = new NettyRpcController();
+
+          try {
+            returnValue = service.callBlockingMethod(methodDescriptor, controller, paramProto);
+          } catch (Throwable t) {
+            throw new RemoteCallException(request.getId(), methodDescriptor, t);
+          }
+
+          RpcResponse.Builder builder = RpcResponse.newBuilder().setId(request.getId());
+
+          if (returnValue != null) {
+            builder.setResponseMessage(returnValue.toByteString());
+          }
+
+          if (controller.failed()) {
+            builder.setErrorMessage(controller.errorText());
+          }
+          ctx.writeAndFlush(builder.build());
+        } finally {
+          ReferenceCountUtil.release(msg);
+        }
+      }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+      if (cause instanceof RemoteCallException) {
+        RemoteCallException callException = (RemoteCallException) cause;
+        ctx.writeAndFlush(callException.getResponse());
+      }
+      
+      if (ctx != null && ctx.channel().isActive()) {
+        ctx.channel().close();
+      }
+    }
+    
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/CallFuture.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/CallFuture.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/CallFuture.java
new file mode 100644
index 0000000..c4c3256
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/CallFuture.java
@@ -0,0 +1,84 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class CallFuture<T> implements RpcCallback<T>, Future<T> {
+
+  private final Semaphore sem = new Semaphore(0);
+  private boolean done = false;
+  private T response;
+  private RpcController controller;
+
+  public CallFuture() {
+    controller = new DefaultRpcController();
+  }
+
+  public RpcController getController() {
+    return controller;
+  }
+
+  @Override
+  public void run(T t) {
+    this.response = t;
+    done = true;
+    sem.release();
+  }
+
+  @Override
+  public boolean cancel(boolean mayInterruptIfRunning) {
+    controller.startCancel();
+    sem.release();
+    return controller.isCanceled();
+  }
+
+  @Override
+  public boolean isCancelled() {
+    return controller.isCanceled();
+  }
+
+  @Override
+  public boolean isDone() {
+    return done;
+  }
+
+  @Override
+  public T get() throws InterruptedException {
+    sem.acquire();
+
+    return response;
+  }
+
+  @Override
+  public T get(long timeout, TimeUnit unit)
+      throws InterruptedException, TimeoutException {
+    if (sem.tryAcquire(timeout, unit)) {
+      return response;
+    } else {
+      throw new TimeoutException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java
new file mode 100644
index 0000000..4ba19a5
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java
@@ -0,0 +1,65 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+
+public class DefaultRpcController implements RpcController {
+  private String errorText;
+  private boolean error;
+  private boolean canceled;
+
+  @Override
+  public void reset() {
+    errorText = "";
+    error = false;
+    canceled = false;
+  }
+
+  @Override
+  public boolean failed() {
+    return error;
+  }
+
+  @Override
+  public String errorText() {
+    return errorText;
+  }
+
+  @Override
+  public void startCancel() {
+    this.canceled = true;
+  }
+
+  @Override
+  public void setFailed(String s) {
+    this.errorText = s;
+    this.error = true;
+  }
+
+  @Override
+  public boolean isCanceled() {
+    return canceled;
+  }
+
+  @Override
+  public void notifyOnCancel(RpcCallback<Object> objectRpcCallback) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyClientBase.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
new file mode 100644
index 0000000..72278f2
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
@@ -0,0 +1,221 @@
+/**
+ * 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.rpc;
+
+import io.netty.channel.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import java.io.Closeable;
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public abstract class NettyClientBase implements Closeable {
+  private static final Log LOG = LogFactory.getLog(NettyClientBase.class);
+  private static final int CONNECTION_TIMEOUT = 60000;  // 60 sec
+  private static final long PAUSE = 1000; // 1 sec
+
+  private final int numRetries;
+
+  private Bootstrap bootstrap;
+  private volatile ChannelFuture channelFuture;
+
+  protected final Class<?> protocol;
+  protected final AtomicInteger sequence = new AtomicInteger(0);
+
+  private final RpcConnectionKey key;
+  private final AtomicInteger counter = new AtomicInteger(0);   // reference counter
+
+  public NettyClientBase(RpcConnectionKey rpcConnectionKey, int numRetries)
+      throws ClassNotFoundException, NoSuchMethodException {
+    this.key = rpcConnectionKey;
+    this.protocol = rpcConnectionKey.protocolClass;
+    this.numRetries = numRetries;
+  }
+
+  // should be called from sub class
+  protected void init(ChannelInitializer<Channel> initializer) {
+    this.bootstrap = new Bootstrap();
+    this.bootstrap
+      .channel(NioSocketChannel.class)
+      .handler(initializer)
+      .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
+      .option(ChannelOption.SO_REUSEADDR, true)
+      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, CONNECTION_TIMEOUT)
+      .option(ChannelOption.SO_RCVBUF, 1048576 * 10)
+      .option(ChannelOption.TCP_NODELAY, true);
+  }
+
+  public RpcConnectionPool.RpcConnectionKey getKey() {
+    return key;
+  }
+
+  protected final Class<?> getServiceClass() throws ClassNotFoundException {
+    String serviceClassName = protocol.getName() + "$" + protocol.getSimpleName() + "Service";
+    return Class.forName(serviceClassName);
+  }
+
+  @SuppressWarnings("unchecked")
+  protected final <T> T getStub(Method stubMethod, Object rpcChannel) {
+    try {
+      return (T) stubMethod.invoke(null, rpcChannel);
+    } catch (Exception e) {
+      throw new RemoteException(e.getMessage(), e);
+    }
+  }
+
+  public abstract <T> T getStub();
+
+  public boolean acquire(long timeout) {
+    if (!checkConnection(timeout)) {
+      return false;
+    }
+    counter.incrementAndGet();
+    return true;
+  }
+
+  public boolean release() {
+    return counter.decrementAndGet() == 0;
+  }
+
+  private boolean checkConnection(long timeout) {
+    if (isConnected()) {
+      return true;
+    }
+
+    InetSocketAddress addr = key.addr;
+    if (addr.isUnresolved()) {
+      addr = RpcUtils.createSocketAddr(addr.getHostName(), addr.getPort());
+    }
+
+    return handleConnectionInternally(addr, timeout);
+  }
+
+  private void connectUsingNetty(InetSocketAddress address, GenericFutureListener<ChannelFuture> listener) {
+    LOG.warn("Try to connect : " + address);
+    this.channelFuture = bootstrap.clone().group(RpcChannelFactory.getSharedClientEventloopGroup())
+            .connect(address)
+            .addListener(listener);
+  }
+
+  // first attendant kicks connection
+  private final RpcUtils.Scrutineer<CountDownLatch> connect = new RpcUtils.Scrutineer<CountDownLatch>();
+
+  private boolean handleConnectionInternally(final InetSocketAddress addr, long timeout) {
+    final CountDownLatch ticket = new CountDownLatch(1);
+    final CountDownLatch granted = connect.check(ticket);
+
+    // basically, it's double checked lock
+    if (ticket == granted && isConnected()) {
+      granted.countDown();
+      return true;
+    }
+
+    if (ticket == granted) {
+      connectUsingNetty(addr, new RetryConnectionListener(addr, granted));
+    }
+
+    try {
+      granted.await(timeout, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      // ignore
+    }
+
+    boolean success = channelFuture.isSuccess();
+
+    if (granted.getCount() == 0) {
+      connect.clear(granted);
+    }
+
+    return success;
+  }
+
+  class RetryConnectionListener implements GenericFutureListener<ChannelFuture> {
+    private final AtomicInteger retryCount = new AtomicInteger();
+    private final InetSocketAddress address;
+    private final CountDownLatch latch;
+
+    RetryConnectionListener(InetSocketAddress address, CountDownLatch latch) {
+      this.address = address;
+      this.latch = latch;
+    }
+
+    @Override
+    public void operationComplete(ChannelFuture channelFuture) throws Exception {
+      if (!channelFuture.isSuccess()) {
+        channelFuture.channel().close();
+
+        if (numRetries > retryCount.getAndIncrement()) {
+          final GenericFutureListener<ChannelFuture> currentListener = this;
+
+          RpcChannelFactory.getSharedClientEventloopGroup().schedule(new Runnable() {
+            @Override
+            public void run() {
+              connectUsingNetty(address, currentListener);
+            }
+          }, PAUSE, TimeUnit.MILLISECONDS);
+
+          LOG.debug("Connecting to " + address + " has been failed. Retrying to connect.");
+        }
+        else {
+          latch.countDown();
+
+          LOG.error("Max retry count has been exceeded. attempts=" + numRetries);
+        }
+      }
+      else {
+        latch.countDown();
+      }
+    }
+  }
+
+  public Channel getChannel() {
+    return channelFuture == null ? null : channelFuture.channel();
+  }
+
+  public boolean isConnected() {
+    Channel channel = getChannel();
+    return channel != null && channel.isOpen() && channel.isActive();
+  }
+
+  public SocketAddress getRemoteAddress() {
+    Channel channel = getChannel();
+    return channel == null ? null : channel.remoteAddress();
+  }
+
+  @Override
+  public void close() {
+    Channel channel = getChannel();
+    if (channel != null && channel.isOpen()) {
+      LOG.debug("Proxy will be disconnected from remote " + channel.remoteAddress());
+      channel.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyRpcController.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyRpcController.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyRpcController.java
new file mode 100644
index 0000000..b7f4537
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyRpcController.java
@@ -0,0 +1,63 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+
+public class NettyRpcController implements RpcController {
+  private String errorText;
+
+  @Override
+  public void reset() {
+    errorText = null;
+  }
+
+  @Override
+  public boolean failed() {
+    return errorText != null;
+  }
+
+  @Override
+  public String errorText() {
+    return errorText;
+  }
+
+  @Override
+  public void startCancel() {
+    // TODO - to be implemented
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setFailed(String s) {
+    errorText = s;
+  }
+
+  @Override
+  public boolean isCanceled() {
+    // TODO - to be implemented
+    return false;
+  }
+
+  @Override
+  public void notifyOnCancel(RpcCallback<Object> objectRpcCallback) {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NullCallback.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NullCallback.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NullCallback.java
new file mode 100644
index 0000000..9b7f8ac
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NullCallback.java
@@ -0,0 +1,38 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.RpcCallback;
+
+public class NullCallback implements RpcCallback<Object> {
+  private final static NullCallback instance;
+
+  static {
+    instance = new NullCallback();
+  }
+
+  public static RpcCallback get() {
+    return instance;
+  }
+
+  @Override
+  public void run(Object parameter) {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.java
new file mode 100644
index 0000000..6a340dc
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.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.rpc;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.handler.codec.protobuf.ProtobufDecoder;
+import io.netty.handler.codec.protobuf.ProtobufEncoder;
+import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
+
+import com.google.protobuf.MessageLite;
+
+class ProtoChannelInitializer extends ChannelInitializer<Channel> {
+  private final MessageLite defaultInstance;
+  private final ChannelHandler handler;
+
+  public ProtoChannelInitializer(ChannelHandler handler, MessageLite defaultInstance) {
+    this.handler = handler;
+    this.defaultInstance = defaultInstance;
+  }
+
+  @Override
+  protected void initChannel(Channel channel) throws Exception {
+    ChannelPipeline pipeline = channel.pipeline();
+    pipeline.addLast("frameDecoder", new ProtobufVarint32FrameDecoder());
+    pipeline.addLast("protobufDecoder", new ProtobufDecoder(defaultInstance));
+    pipeline.addLast("frameEncoder", new ProtobufVarint32LengthFieldPrepender());
+    pipeline.addLast("protobufEncoder", new ProtobufEncoder());
+    pipeline.addLast("handler", handler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteCallException.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
new file mode 100644
index 0000000..52ef31a
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
@@ -0,0 +1,69 @@
+/**
+ * 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.rpc;
+
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import org.apache.tajo.rpc.RpcProtos.RpcResponse;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.io.Writer;
+
+public class RemoteCallException extends RemoteException {
+  private int seqId;
+  private String originExceptionClass;
+
+  public RemoteCallException(int seqId, MethodDescriptor methodDesc,
+                             Throwable t) {
+    super("Remote call error occurs when " + methodDesc.getFullName() + "is called:", t);
+    this.seqId = seqId;
+    if (t != null) {
+      originExceptionClass = t.getClass().getCanonicalName();
+    }
+  }
+
+  public RemoteCallException(int seqId, Throwable t) {
+    super(t);
+    this.seqId = seqId;
+    if (t != null) {
+      originExceptionClass = t.getClass().getCanonicalName();
+    }
+  }
+
+  public RpcResponse getResponse() {
+    RpcResponse.Builder builder = RpcResponse.newBuilder();
+    builder.setId(seqId);
+    if (getCause().getMessage() == null) {
+      builder.setErrorMessage(getCause().getClass().getName());
+    } else {
+      builder.setErrorMessage(getCause().getMessage());
+    }
+    builder.setErrorTrace(getStackTraceString(getCause()));
+    builder.setErrorClass(originExceptionClass);
+
+    return builder.build();
+  }
+
+  private static String getStackTraceString(Throwable aThrowable) {
+    final Writer result = new StringWriter();
+    final PrintWriter printWriter = new PrintWriter(result);
+    aThrowable.printStackTrace(printWriter);
+    return result.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteException.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteException.java
new file mode 100644
index 0000000..30c110d
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RemoteException.java
@@ -0,0 +1,37 @@
+/**
+ * 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.rpc;
+
+public class RemoteException extends RuntimeException {
+  public RemoteException() {
+    super();
+  }
+
+  public RemoteException(String message) {
+    super(message);
+  }
+
+  public RemoteException(Throwable t) {
+    super(t);
+  }
+
+  public RemoteException(String message, Throwable t) {
+    super(message, t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
new file mode 100644
index 0000000..3c054ad
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
@@ -0,0 +1,104 @@
+/**
+ * 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.rpc;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.List;
+
+public class RetriesExhaustedException extends RuntimeException {
+  private static final long serialVersionUID = 1876775844L;
+
+  public RetriesExhaustedException(final String msg) {
+    super(msg);
+  }
+
+  public RetriesExhaustedException(final String msg, final IOException e) {
+    super(msg, e);
+  }
+
+  /**
+   * Datastructure that allows adding more info around Throwable incident.
+   */
+  public static class ThrowableWithExtraContext {
+    private final Throwable t;
+    private final long when;
+    private final String extras;
+
+    public ThrowableWithExtraContext(final Throwable t, final long when,
+        final String extras) {
+      this.t = t;
+      this.when = when;
+      this.extras = extras;
+    }
+ 
+    @Override
+    public String toString() {
+      return new Date(this.when).toString() + ", " + extras + ", " + t.toString();
+    }
+  }
+
+  /**
+   * Create a new RetriesExhaustedException from the list of prior failures.
+   * @param callableVitals Details from the {@link ServerCallable} we were using
+   * when we got this exception.
+   * @param numTries The number of tries we made
+   * @param exceptions List of exceptions that failed before giving up
+   */
+  public RetriesExhaustedException(final String callableVitals, int numTries,
+      List<Throwable> exceptions) {
+    super(getMessage(callableVitals, numTries, exceptions));
+  }
+
+  /**
+   * Create a new RetriesExhaustedException from the list of prior failures.
+   * @param numTries
+   * @param exceptions List of exceptions that failed before giving up
+   */
+  public RetriesExhaustedException(final int numTries,
+      final List<Throwable> exceptions) {
+    super(getMessage(numTries, exceptions));
+  }
+
+  private static String getMessage(String callableVitals, int numTries,
+      List<Throwable> exceptions) {
+    StringBuilder buffer = new StringBuilder("Failed contacting ");
+    buffer.append(callableVitals);
+    buffer.append(" after ");
+    buffer.append(numTries + 1);
+    buffer.append(" attempts.\nExceptions:\n");
+    for (Throwable t : exceptions) {
+      buffer.append(t.toString());
+      buffer.append("\n");
+    }
+    return buffer.toString();
+  }
+
+  private static String getMessage(final int numTries,
+      final List<Throwable> exceptions) {
+    StringBuilder buffer = new StringBuilder("Failed after attempts=");
+    buffer.append(numTries + 1);
+    buffer.append(", exceptions:\n");
+    for (Throwable t : exceptions) {
+      buffer.append(t.toString());
+      buffer.append("\n");
+    }
+    return buffer.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
new file mode 100644
index 0000000..6d1f479
--- /dev/null
+++ b/tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
@@ -0,0 +1,190 @@
+/**
+ * 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.rpc;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import io.netty.channel.ConnectTimeoutException;
+import io.netty.util.internal.logging.CommonsLoggerFactory;
+import io.netty.util.internal.logging.InternalLoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+public class RpcConnectionPool {
+  private static final Log LOG = LogFactory.getLog(RpcConnectionPool.class);
+
+  private Map<RpcConnectionKey, NettyClientBase> connections =
+      new HashMap<RpcConnectionKey, NettyClientBase>();
+
+  private static RpcConnectionPool instance;
+  private final Object lockObject = new Object();
+
+  public final static int RPC_RETRIES = 3;
+
+  private RpcConnectionPool() {
+  }
+
+  public synchronized static RpcConnectionPool getPool() {
+    if(instance == null) {
+      InternalLoggerFactory.setDefaultFactory(new CommonsLoggerFactory());
+      instance = new RpcConnectionPool();
+    }
+    return instance;
+  }
+
+  private NettyClientBase makeConnection(RpcConnectionKey rpcConnectionKey)
+      throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
+    NettyClientBase client;
+    if(rpcConnectionKey.asyncMode) {
+      client = new AsyncRpcClient(rpcConnectionKey, RPC_RETRIES);
+    } else {
+      client = new BlockingRpcClient(rpcConnectionKey, RPC_RETRIES);
+    }
+    return client;
+  }
+
+  public static final long DEFAULT_TIMEOUT = 3000;
+  public static final long DEFAULT_INTERVAL = 500;
+
+  public NettyClientBase getConnection(InetSocketAddress addr,
+                                       Class<?> protocolClass, boolean asyncMode)
+      throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
+    return getConnection(addr, protocolClass, asyncMode, DEFAULT_TIMEOUT, DEFAULT_INTERVAL);
+  }
+
+  public NettyClientBase getConnection(InetSocketAddress addr,
+      Class<?> protocolClass, boolean asyncMode, long timeout, long interval)
+      throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
+    RpcConnectionKey key = new RpcConnectionKey(addr, protocolClass, asyncMode);
+
+    RpcUtils.Timer timer = new RpcUtils.Timer(timeout);
+    for (; !timer.isTimedOut(); timer.elapsed()) {
+      NettyClientBase client;
+      synchronized (lockObject) {
+        client = connections.get(key);
+        if (client == null) {
+          connections.put(key, client = makeConnection(key));
+        }
+      }
+      if (client.acquire(timer.remaining())) {
+        return client;
+      }
+      timer.interval(interval);
+    }
+
+    throw new ConnectTimeoutException("Failed to get connection for " + timeout + " msec");
+  }
+
+  public void releaseConnection(NettyClientBase client) {
+    release(client, false);
+  }
+
+  public void closeConnection(NettyClientBase client) {
+    release(client, true);
+  }
+
+  private void release(NettyClientBase client, boolean close) {
+    if (client == null) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Close connection [" + client.getKey() + "]");
+    }
+    try {
+      if (returnToPool(client, close)) {
+        client.close();
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Current Connections [" + connections.size() + "]");
+      }
+    } catch (Exception e) {
+      LOG.error("Can't close connection:" + client.getKey() + ":" + e.getMessage(), e);
+    }
+  }
+
+  // return true if the connection should be closed
+  private boolean returnToPool(NettyClientBase client, boolean close) {
+    synchronized (lockObject) {
+      if (client.release() && (close || !client.isConnected())) {
+        connections.remove(client.getKey());
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public void close() {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Pool Closed");
+    }
+
+    synchronized (lockObject) {
+      for (NettyClientBase eachClient : connections.values()) {
+        try {
+          eachClient.close();
+        } catch (Exception e) {
+          LOG.error("close client pool error", e);
+        }
+      }
+      connections.clear();
+    }
+  }
+
+  public void shutdown(){
+    close();
+    RpcChannelFactory.shutdownGracefully();
+  }
+
+  static class RpcConnectionKey {
+    final InetSocketAddress addr;
+    final Class<?> protocolClass;
+    final boolean asyncMode;
+
+    final String description;
+
+    public RpcConnectionKey(InetSocketAddress addr,
+                            Class<?> protocolClass, boolean asyncMode) {
+      this.addr = addr;
+      this.protocolClass = protocolClass;
+      this.asyncMode = asyncMode;
+      this.description = "["+ protocolClass + "] " + addr + "," + asyncMode;
+    }
+
+    @Override
+    public String toString() {
+      return description;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if(!(obj instanceof RpcConnectionKey)) {
+        return false;
+      }
+
+      return toString().equals(obj.toString());
+    }
+
+    @Override
+    public int hashCode() {
+      return description.hashCode();
+    }
+  }
+}


[04/13] tajo git commit: TAJO-1391: RpcConnectionPool should check reference counter of connection before close

Posted by ji...@apache.org.
TAJO-1391: RpcConnectionPool should check reference counter of connection before close

Closes #412

Signed-off-by: Jihun Kang <ji...@apache.org>


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

Branch: refs/heads/index_support
Commit: 0dc7d68071dcf7c9d01dde8ed7598ca422e4c50c
Parents: e1e38e2
Author: navis.ryu <na...@apache.org>
Authored: Mon Mar 16 10:03:10 2015 +0900
Committer: Jihun Kang <ji...@apache.org>
Committed: Mon Mar 16 10:03:10 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../apache/tajo/client/SessionConnection.java   |   2 +-
 .../org/apache/tajo/rpc/AsyncRpcClient.java     |  63 ++------
 .../org/apache/tajo/rpc/BlockingRpcClient.java  |  68 ++-------
 .../org/apache/tajo/rpc/NettyClientBase.java    | 148 ++++++++++++-------
 .../org/apache/tajo/rpc/RpcConnectionPool.java  | 112 +++++++-------
 .../main/java/org/apache/tajo/rpc/RpcUtils.java |  54 +++++++
 .../java/org/apache/tajo/rpc/TestAsyncRpc.java  |  32 ++--
 .../org/apache/tajo/rpc/TestBlockingRpc.java    |  34 +++--
 9 files changed, 272 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 84a7571..9d2cd14 100644
--- a/CHANGES
+++ b/CHANGES
@@ -9,6 +9,9 @@ Release 0.11.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1391: RpcConnectionPool should check reference counter of connection 
+    before close. (Contributed by navis, Committed by jihun)
+
     TAJO-1383: Improve broadcast table cache. (jinho)
 
     TAJO-1374: Support multi-bytes delimiter for CSV file.

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
index d05d3b1..d24e7b3 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
@@ -128,7 +128,7 @@ public class SessionConnection implements Closeable {
     if(!closed.get()){
       try {
         return connPool.getConnection(serviceTracker.getClientServiceAddress(),
-            TajoMasterClientProtocol.class, false).isActive();
+            TajoMasterClientProtocol.class, false).isConnected();
       } catch (Throwable e) {
         return false;
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
index 5845229..1ea9fb1 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
@@ -24,6 +24,7 @@ import com.google.protobuf.*;
 import io.netty.channel.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
 import org.apache.tajo.rpc.RpcProtos.RpcRequest;
 import org.apache.tajo.rpc.RpcProtos.RpcResponse;
 
@@ -34,62 +35,33 @@ import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
 
 public class AsyncRpcClient extends NettyClientBase {
   private static final Log LOG = LogFactory.getLog(AsyncRpcClient.class);
 
-  private final ChannelInitializer<Channel> initializer;
-  private final ProxyRpcChannel rpcChannel;
-
-  private final AtomicInteger sequence = new AtomicInteger(0);
   private final Map<Integer, ResponseCallback> requests =
       new ConcurrentHashMap<Integer, ResponseCallback>();
 
-  private final Class<?> protocol;
   private final Method stubMethod;
-
-  private RpcConnectionKey key;
+  private final ProxyRpcChannel rpcChannel;
+  private final ClientChannelInboundHandler inboundHandler;
 
   /**
    * Intentionally make this method package-private, avoiding user directly
    * new an instance through this constructor.
    */
-  AsyncRpcClient(final Class<?> protocol,
-                        final InetSocketAddress addr, int retries)
-      throws ClassNotFoundException, NoSuchMethodException, ConnectTimeoutException {
-
-    this.protocol = protocol;
-    String serviceClassName = protocol.getName() + "$"
-        + protocol.getSimpleName() + "Service";
-    Class<?> serviceClass = Class.forName(serviceClassName);
-    stubMethod = serviceClass.getMethod("newStub", RpcChannel.class);
-
-    initializer = new ProtoChannelInitializer(new ClientChannelInboundHandler(), 
-        RpcResponse.getDefaultInstance());
-    super.init(addr, initializer, retries);
+  AsyncRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
+      throws ClassNotFoundException, NoSuchMethodException {
+    super(rpcConnectionKey, retries);
+    stubMethod = getServiceClass().getMethod("newStub", RpcChannel.class);
     rpcChannel = new ProxyRpcChannel();
-    this.key = new RpcConnectionKey(addr, protocol, true);
-  }
-
-  @Override
-  public RpcConnectionKey getKey() {
-    return key;
+    inboundHandler = new ClientChannelInboundHandler();
+    init(new ProtoChannelInitializer(inboundHandler, RpcResponse.getDefaultInstance()));
   }
 
   @Override
   public <T> T getStub() {
-    try {
-      return (T) stubMethod.invoke(null, rpcChannel);
-    } catch (Exception e) {
-      throw new RemoteException(e.getMessage(), e);
-    }
-  }
-
-  public RpcChannel getRpcChannel() {
-    return this.rpcChannel;
+    return getStub(stubMethod, rpcChannel);
   }
 
   protected void sendExceptions(String message) {
@@ -113,17 +85,6 @@ public class AsyncRpcClient extends NettyClientBase {
   }
 
   private class ProxyRpcChannel implements RpcChannel {
-    private final ClientChannelInboundHandler handler;
-
-    public ProxyRpcChannel() {
-      this.handler = getChannel().pipeline()
-          .get(ClientChannelInboundHandler.class);
-
-      if (handler == null) {
-        throw new IllegalArgumentException("Channel does not have " +
-            "proper handler");
-      }
-    }
 
     public void callMethod(final MethodDescriptor method,
                            final RpcController controller,
@@ -135,7 +96,7 @@ public class AsyncRpcClient extends NettyClientBase {
 
       Message rpcRequest = buildRequest(nextSeqId, method, param);
 
-      handler.registerCallback(nextSeqId,
+      inboundHandler.registerCallback(nextSeqId,
           new ResponseCallback(controller, responseType, done));
 
       ChannelPromise channelPromise = getChannel().newPromise();
@@ -144,7 +105,7 @@ public class AsyncRpcClient extends NettyClientBase {
         @Override
         public void operationComplete(ChannelFuture future) throws Exception {
           if (!future.isSuccess()) {
-            handler.exceptionCaught(null, new ServiceException(future.cause()));
+            inboundHandler.exceptionCaught(null, new ServiceException(future.cause()));
           }
         }
       });

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
index 4ec5718..6a90330 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
@@ -25,6 +25,7 @@ import io.netty.channel.*;
 import io.netty.util.concurrent.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
 import org.apache.tajo.rpc.RpcProtos.RpcRequest;
 import org.apache.tajo.rpc.RpcProtos.RpcResponse;
 
@@ -35,63 +36,33 @@ import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.concurrent.*;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
 
 public class BlockingRpcClient extends NettyClientBase {
   private static final Log LOG = LogFactory.getLog(RpcProtos.class);
 
-  private final ChannelInitializer<Channel> initializer;
-  private final ProxyRpcChannel rpcChannel;
-
-  private final AtomicInteger sequence = new AtomicInteger(0);
   private final Map<Integer, ProtoCallFuture> requests =
       new ConcurrentHashMap<Integer, ProtoCallFuture>();
 
-  private final Class<?> protocol;
   private final Method stubMethod;
-
-  private RpcConnectionKey key;
+  private final ProxyRpcChannel rpcChannel;
+  private final ChannelInboundHandlerAdapter inboundHandler;
 
   /**
    * Intentionally make this method package-private, avoiding user directly
    * new an instance through this constructor.
    */
-  BlockingRpcClient(final Class<?> protocol,
-                           final InetSocketAddress addr, int retries)
-      throws ClassNotFoundException, NoSuchMethodException, ConnectTimeoutException {
-
-    this.protocol = protocol;
-    String serviceClassName = protocol.getName() + "$"
-        + protocol.getSimpleName() + "Service";
-    Class<?> serviceClass = Class.forName(serviceClassName);
-    stubMethod = serviceClass.getMethod("newBlockingStub",
-        BlockingRpcChannel.class);
-
-    initializer = new ProtoChannelInitializer(new ClientChannelInboundHandler(), RpcResponse.getDefaultInstance());
-    super.init(addr, initializer, retries);
+  BlockingRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
+      throws ClassNotFoundException, NoSuchMethodException {
+    super(rpcConnectionKey, retries);
+    stubMethod = getServiceClass().getMethod("newBlockingStub", BlockingRpcChannel.class);
     rpcChannel = new ProxyRpcChannel();
-
-    this.key = new RpcConnectionKey(addr, protocol, false);
-  }
-
-  @Override
-  public RpcConnectionKey getKey() {
-    return key;
+    inboundHandler = new ClientChannelInboundHandler();
+    init(new ProtoChannelInitializer(inboundHandler, RpcResponse.getDefaultInstance()));
   }
 
   @Override
   public <T> T getStub() {
-    try {
-      return (T) stubMethod.invoke(null, rpcChannel);
-    } catch (Exception e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-  }
-
-  public BlockingRpcChannel getBlockingRpcChannel() {
-    return this.rpcChannel;
+    return getStub(stubMethod, rpcChannel);
   }
 
   @Override
@@ -106,19 +77,6 @@ public class BlockingRpcClient extends NettyClientBase {
 
   private class ProxyRpcChannel implements BlockingRpcChannel {
 
-    private final ClientChannelInboundHandler handler;
-
-    public ProxyRpcChannel() {
-
-      this.handler = getChannel().pipeline().
-          get(ClientChannelInboundHandler.class);
-
-      if (handler == null) {
-        throw new IllegalArgumentException("Channel does not have " +
-            "proper handler");
-      }
-    }
-
     @Override
     public Message callBlockingMethod(final MethodDescriptor method,
                                       final RpcController controller,
@@ -139,7 +97,7 @@ public class BlockingRpcClient extends NettyClientBase {
         @Override
         public void operationComplete(ChannelFuture future) throws Exception {
           if (!future.isSuccess()) {
-            handler.exceptionCaught(null, new ServiceException(future.cause()));
+            inboundHandler.exceptionCaught(null, new ServiceException(future.cause()));
           }
         }
       });
@@ -174,7 +132,7 @@ public class BlockingRpcClient extends NettyClientBase {
   }
 
   private String getErrorMessage(String message) {
-    if(protocol != null && getChannel() != null) {
+    if(getChannel() != null) {
       return protocol.getName() +
           "(" + RpcUtils.normalizeInetSocketAddress((InetSocketAddress)
           getChannel().remoteAddress()) + "): " + message;
@@ -184,7 +142,7 @@ public class BlockingRpcClient extends NettyClientBase {
   }
 
   private TajoServiceException makeTajoServiceException(RpcResponse response, Throwable cause) {
-    if(protocol != null && getChannel() != null) {
+    if(getChannel() != null) {
       return new TajoServiceException(response.getErrorMessage(), cause, protocol.getName(),
           RpcUtils.normalizeInetSocketAddress((InetSocketAddress)getChannel().remoteAddress()));
     } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
index 7b52178..7dfc5a2 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
@@ -20,9 +20,9 @@ package org.apache.tajo.rpc;
 
 import io.netty.channel.*;
 
-import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
 
 import io.netty.bootstrap.Bootstrap;
 import io.netty.buffer.PooledByteBufAllocator;
@@ -30,77 +30,125 @@ import io.netty.channel.socket.nio.NioSocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
 
 import java.io.Closeable;
+import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 public abstract class NettyClientBase implements Closeable {
-  private static Log LOG = LogFactory.getLog(NettyClientBase.class);
-  private static final int CLIENT_CONNECTION_TIMEOUT_SEC = 60;
+  private static final Log LOG = LogFactory.getLog(NettyClientBase.class);
+  private static final int CONNECTION_TIMEOUT = 60000;  // 60 sec
   private static final long PAUSE = 1000; // 1 sec
-  private int numRetries;
 
-  protected Bootstrap bootstrap;
-  private ChannelFuture channelFuture;
+  private final int numRetries;
 
-  public NettyClientBase() {
-  }
+  private Bootstrap bootstrap;
+  private volatile ChannelFuture channelFuture;
 
-  public abstract <T> T getStub();
-  public abstract RpcConnectionPool.RpcConnectionKey getKey();
-  
-  public void init(InetSocketAddress addr, ChannelInitializer<Channel> initializer, 
-      int numRetries) throws ConnectTimeoutException {
+  protected final Class<?> protocol;
+  protected final AtomicInteger sequence = new AtomicInteger(0);
+
+  private final RpcConnectionKey key;
+  private final AtomicInteger counter = new AtomicInteger(0);   // reference counter
+
+  public NettyClientBase(RpcConnectionKey rpcConnectionKey, int numRetries)
+      throws ClassNotFoundException, NoSuchMethodException {
+    this.key = rpcConnectionKey;
+    this.protocol = rpcConnectionKey.protocolClass;
     this.numRetries = numRetries;
-    
-    init(addr, initializer);
   }
 
-  public void init(InetSocketAddress addr, ChannelInitializer<Channel> initializer)
-      throws ConnectTimeoutException {
+  // should be called from sub class
+  protected void init(ChannelInitializer<Channel> initializer) {
     this.bootstrap = new Bootstrap();
     this.bootstrap
       .channel(NioSocketChannel.class)
       .handler(initializer)
       .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
       .option(ChannelOption.SO_REUSEADDR, true)
-      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 10000)
+      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, CONNECTION_TIMEOUT)
       .option(ChannelOption.SO_RCVBUF, 1048576 * 10)
       .option(ChannelOption.TCP_NODELAY, true);
+  }
 
-    connect(addr);
+  public RpcConnectionPool.RpcConnectionKey getKey() {
+    return key;
   }
 
-  private void connectUsingNetty(InetSocketAddress address, GenericFutureListener<ChannelFuture> listener) {
+  protected final Class<?> getServiceClass() throws ClassNotFoundException {
+    String serviceClassName = protocol.getName() + "$" + protocol.getSimpleName() + "Service";
+    return Class.forName(serviceClassName);
+  }
 
+  @SuppressWarnings("unchecked")
+  protected final <T> T getStub(Method stubMethod, Object rpcChannel) {
+    try {
+      return (T) stubMethod.invoke(null, rpcChannel);
+    } catch (Exception e) {
+      throw new RemoteException(e.getMessage(), e);
+    }
+  }
+
+  public abstract <T> T getStub();
+
+  public boolean acquire(long timeout) {
+    if (!checkConnection(timeout)) {
+      return false;
+    }
+    counter.incrementAndGet();
+    return true;
+  }
+
+  public boolean release() {
+    return counter.decrementAndGet() == 0;
+  }
+
+  private boolean checkConnection(long timeout) {
+    if (isConnected()) {
+      return true;
+    }
+
+    InetSocketAddress addr = key.addr;
+    if (addr.isUnresolved()) {
+      addr = RpcUtils.createSocketAddr(addr.getHostName(), addr.getPort());
+    }
+
+    return handleConnectionInternally(addr, timeout);
+  }
+
+  private void connectUsingNetty(InetSocketAddress address, GenericFutureListener<ChannelFuture> listener) {
+    LOG.warn("Try to connect : " + address);
     this.channelFuture = bootstrap.clone().group(RpcChannelFactory.getSharedClientEventloopGroup())
             .connect(address)
             .addListener(listener);
   }
-  
-  private void handleConnectionInternally(final InetSocketAddress addr) throws ConnectTimeoutException {
-    final CountDownLatch latch = new CountDownLatch(1);
-    GenericFutureListener<ChannelFuture> listener = new RetryConnectionListener(addr, latch);
-    connectUsingNetty(addr, listener);
+
+  // first attendant kicks connection
+  private final RpcUtils.Scrutineer<CountDownLatch> connect = new RpcUtils.Scrutineer<CountDownLatch>();
+
+  private boolean handleConnectionInternally(final InetSocketAddress addr, long timeout) {
+    final CountDownLatch ticket = new CountDownLatch(1);
+    final CountDownLatch granted = connect.check(ticket);
+
+    if (ticket == granted) {
+      connectUsingNetty(addr, new RetryConnectionListener(addr, granted));
+    }
 
     try {
-      latch.await(CLIENT_CONNECTION_TIMEOUT_SEC, TimeUnit.SECONDS);
+      granted.await(timeout, TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
+      // ignore
     }
 
-    if (!channelFuture.isSuccess()) {
-      throw new ConnectTimeoutException("Connect error to " + addr +
-          " caused by " + ExceptionUtils.getMessage(channelFuture.cause()));
-    }
-  }
+    boolean success = channelFuture.isSuccess();
 
-  public void connect(InetSocketAddress addr) throws ConnectTimeoutException {
-    if(addr.isUnresolved()){
-       addr = RpcUtils.createSocketAddr(addr.getHostName(), addr.getPort());
+    if (granted.getCount() == 0) {
+      connect.clear(granted);
     }
 
-    handleConnectionInternally(addr);
+    return success;
   }
 
   class RetryConnectionListener implements GenericFutureListener<ChannelFuture> {
@@ -142,32 +190,26 @@ public abstract class NettyClientBase implements Closeable {
     }
   }
 
-  public boolean isActive() {
-    return getChannel().isActive();
+  public Channel getChannel() {
+    return channelFuture == null ? null : channelFuture.channel();
   }
 
-  public InetSocketAddress getRemoteAddress() {
-    if (channelFuture == null || channelFuture.channel() == null) {
-      return null;
-    }
-    return (InetSocketAddress) channelFuture.channel().remoteAddress();
+  public boolean isConnected() {
+    Channel channel = getChannel();
+    return channel != null && channel.isOpen() && channel.isActive();
   }
 
-  public Channel getChannel() {
-    return channelFuture.channel();
+  public SocketAddress getRemoteAddress() {
+    Channel channel = getChannel();
+    return channel == null ? null : channel.remoteAddress();
   }
 
   @Override
   public void close() {
-    if (channelFuture != null && getChannel().isActive()) {
-      getChannel().close();
-    }
-
-    if (this.bootstrap != null) {
-      InetSocketAddress address = getRemoteAddress();
-      if (address != null) {
-        LOG.debug("Proxy is disconnected from " + address.getHostName() + ":" + address.getPort());
-      }
+    Channel channel = getChannel();
+    if (channel != null && channel.isOpen()) {
+      LOG.debug("Proxy will be disconnected from remote " + channel.remoteAddress());
+      channel.close();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
index 43feeb1..6d1f479 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcConnectionPool.java
@@ -18,13 +18,9 @@
 
 package org.apache.tajo.rpc;
 
-import com.google.common.base.Objects;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import io.netty.channel.ConnectTimeoutException;
-import io.netty.channel.group.ChannelGroup;
-import io.netty.channel.group.DefaultChannelGroup;
-import io.netty.util.concurrent.GlobalEventExecutor;
 import io.netty.util.internal.logging.CommonsLoggerFactory;
 import io.netty.util.internal.logging.InternalLoggerFactory;
 
@@ -37,7 +33,6 @@ public class RpcConnectionPool {
 
   private Map<RpcConnectionKey, NettyClientBase> connections =
       new HashMap<RpcConnectionKey, NettyClientBase>();
-  private ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
 
   private static RpcConnectionPool instance;
   private final Object lockObject = new Object();
@@ -59,103 +54,101 @@ public class RpcConnectionPool {
       throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
     NettyClientBase client;
     if(rpcConnectionKey.asyncMode) {
-      client = new AsyncRpcClient(rpcConnectionKey.protocolClass, rpcConnectionKey.addr, 
-          RPC_RETRIES);
+      client = new AsyncRpcClient(rpcConnectionKey, RPC_RETRIES);
     } else {
-      client = new BlockingRpcClient(rpcConnectionKey.protocolClass, rpcConnectionKey.addr, 
-          RPC_RETRIES);
+      client = new BlockingRpcClient(rpcConnectionKey, RPC_RETRIES);
     }
-    accepted.add(client.getChannel());
     return client;
   }
 
+  public static final long DEFAULT_TIMEOUT = 3000;
+  public static final long DEFAULT_INTERVAL = 500;
+
   public NettyClientBase getConnection(InetSocketAddress addr,
                                        Class<?> protocolClass, boolean asyncMode)
       throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
+    return getConnection(addr, protocolClass, asyncMode, DEFAULT_TIMEOUT, DEFAULT_INTERVAL);
+  }
+
+  public NettyClientBase getConnection(InetSocketAddress addr,
+      Class<?> protocolClass, boolean asyncMode, long timeout, long interval)
+      throws NoSuchMethodException, ClassNotFoundException, ConnectTimeoutException {
     RpcConnectionKey key = new RpcConnectionKey(addr, protocolClass, asyncMode);
-    NettyClientBase client = connections.get(key);
 
-    if (client == null) {
-      synchronized (lockObject){
+    RpcUtils.Timer timer = new RpcUtils.Timer(timeout);
+    for (; !timer.isTimedOut(); timer.elapsed()) {
+      NettyClientBase client;
+      synchronized (lockObject) {
         client = connections.get(key);
         if (client == null) {
-          client = makeConnection(key);
-          connections.put(key, client);
+          connections.put(key, client = makeConnection(key));
         }
       }
+      if (client.acquire(timer.remaining())) {
+        return client;
+      }
+      timer.interval(interval);
     }
 
-    if (client.getChannel() == null || !client.getChannel().isOpen() || !client.getChannel().isActive()) {
-      LOG.warn("Try to reconnect : " + addr);
-      client.connect(addr);
-    }
-    return client;
+    throw new ConnectTimeoutException("Failed to get connection for " + timeout + " msec");
   }
 
   public void releaseConnection(NettyClientBase client) {
-    if (client == null) return;
-
-    try {
-      synchronized (lockObject) {
-        if (!client.getChannel().isOpen()) {
-          connections.remove(client.getKey());
-          client.close();
-        }
-      }
-
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Current Connections [" + connections.size() + "] Accepted: " + accepted.size());
-
-      }
-    } catch (Exception e) {
-      LOG.error("Can't close connection:" + client.getKey() + ":" + e.getMessage(), e);
-    }
+    release(client, false);
   }
 
   public void closeConnection(NettyClientBase client) {
+    release(client, true);
+  }
+
+  private void release(NettyClientBase client, boolean close) {
     if (client == null) {
       return;
     }
-
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Close connection [" + client.getKey() + "]");
+    }
     try {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Close connection [" + client.getKey() + "]");
-      }
-
-      synchronized (lockObject) {
-        connections.remove(client.getKey());
+      if (returnToPool(client, close)) {
         client.close();
       }
-
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Current Connections [" + connections.size() + "]");
+      }
     } catch (Exception e) {
       LOG.error("Can't close connection:" + client.getKey() + ":" + e.getMessage(), e);
     }
   }
 
-  public synchronized void close() {
+  // return true if the connection should be closed
+  private boolean returnToPool(NettyClientBase client, boolean close) {
+    synchronized (lockObject) {
+      if (client.release() && (close || !client.isConnected())) {
+        connections.remove(client.getKey());
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public void close() {
     if(LOG.isDebugEnabled()) {
       LOG.debug("Pool Closed");
     }
-    synchronized(lockObject) {
-      for(NettyClientBase eachClient: connections.values()) {
+
+    synchronized (lockObject) {
+      for (NettyClientBase eachClient : connections.values()) {
         try {
           eachClient.close();
         } catch (Exception e) {
           LOG.error("close client pool error", e);
         }
       }
-
       connections.clear();
     }
-
-    try {
-      accepted.close();
-    } catch (Throwable t) {
-      LOG.error(t, t);
-    }
   }
 
-  public synchronized void shutdown(){
+  public void shutdown(){
     close();
     RpcChannelFactory.shutdownGracefully();
   }
@@ -165,16 +158,19 @@ public class RpcConnectionPool {
     final Class<?> protocolClass;
     final boolean asyncMode;
 
+    final String description;
+
     public RpcConnectionKey(InetSocketAddress addr,
                             Class<?> protocolClass, boolean asyncMode) {
       this.addr = addr;
       this.protocolClass = protocolClass;
       this.asyncMode = asyncMode;
+      this.description = "["+ protocolClass + "] " + addr + "," + asyncMode;
     }
 
     @Override
     public String toString() {
-      return "["+ protocolClass + "] " + addr + "," + asyncMode;
+      return description;
     }
 
     @Override
@@ -188,7 +184,7 @@ public class RpcConnectionPool {
 
     @Override
     public int hashCode() {
-      return Objects.hashCode(addr, asyncMode);
+      return description.hashCode();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java
index b6be05f..152d426 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RpcUtils.java
@@ -21,6 +21,7 @@ package org.apache.tajo.rpc;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicReference;
 
 public class RpcUtils {
 
@@ -65,4 +66,57 @@ public class RpcUtils {
     String [] splitted = addr.split(":");
     return InetSocketAddress.createUnresolved(splitted[0], Integer.parseInt(splitted[1]));
   }
+
+  public static class Timer {
+    private long remaining;
+    private long prev;
+    public Timer(long timeout) {
+      this.remaining = timeout;
+      this.prev = System.currentTimeMillis();
+    }
+
+    public boolean isTimedOut() {
+      return remaining <= 0;
+    }
+
+    public void elapsed() {
+      long current = System.currentTimeMillis();
+      remaining -= (prev - current);
+      prev = current;
+    }
+
+    public void interval(long wait) {
+      if (wait <= 0 || isTimedOut()) {
+        return;
+      }
+      try {
+        Thread.sleep(Math.min(remaining, wait));
+      } catch (Exception ex) {
+        // ignore
+      }
+    }
+
+    public long remaining() {
+      return remaining;
+    }
+  }
+
+  public static class Scrutineer<T> {
+
+    private final AtomicReference<T> reference = new AtomicReference<T>();
+
+    T check(T ticket) {
+      T granted = reference.get();
+      for (;granted == null; granted = reference.get()) {
+        if (reference.compareAndSet(null, ticket)) {
+          return ticket;
+        }
+      }
+      return granted;
+    }
+
+    boolean clear(T granted) {
+      return reference.compareAndSet(granted, null);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
index 31d5265..a974a65 100644
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
+++ b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestAsyncRpc.java
@@ -34,8 +34,6 @@ import org.junit.rules.ExternalResource;
 import org.junit.runner.Description;
 import org.junit.runners.model.Statement;
 
-import io.netty.channel.ConnectTimeoutException;
-
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
@@ -125,8 +123,12 @@ public class TestAsyncRpc {
   public void setUpRpcClient() throws Exception {
     retries = 1;
 
-    client = new AsyncRpcClient(DummyProtocol.class,
-        RpcUtils.getConnectAddress(server.getListenAddress()), retries);
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.getConnectAddress(server.getListenAddress()),
+              DummyProtocol.class, true);
+    client = new AsyncRpcClient(rpcConnectionKey, retries);
+    client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT);
     stub = client.getStub();
   }
 
@@ -296,7 +298,10 @@ public class TestAsyncRpc {
     });
     serverThread.start();
 
-    client = new AsyncRpcClient(DummyProtocol.class, address, retries);
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, true);
+    client = new AsyncRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
     stub = client.getStub();
     stub.echo(future.getController(), echoMessage, future);
 
@@ -308,24 +313,25 @@ public class TestAsyncRpc {
   @Test
   public void testConnectionFailure() throws Exception {
     InetSocketAddress address = new InetSocketAddress("test", 0);
-    boolean expected = false;
     try {
-      new AsyncRpcClient(DummyProtocol.class, address, retries);
-      fail();
-    } catch (ConnectTimeoutException e) {
-      expected = true;
+      RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, true);
+      NettyClientBase client = new AsyncRpcClient(rpcConnectionKey, retries);
+      assertFalse(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
     } catch (Throwable throwable) {
       fail();
     }
-    assertTrue(expected);
   }
 
   @Test
   @SetupRpcConnection(setupRpcClient=false)
   public void testUnresolvedAddress() throws Exception {
     String hostAndPort = RpcUtils.normalizeInetSocketAddress(server.getListenAddress());
-    client = new AsyncRpcClient(DummyProtocol.class,
-        RpcUtils.createUnresolved(hostAndPort), retries);
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.createUnresolved(hostAndPort), DummyProtocol.class, true);
+    client = new AsyncRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
     Interface stub = client.getStub();
     EchoMessage echoMessage = EchoMessage.newBuilder()
         .setMessage(MESSAGE).build();

http://git-wip-us.apache.org/repos/asf/tajo/blob/0dc7d680/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
index 07e2dca..10dd766 100644
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
+++ b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestBlockingRpc.java
@@ -35,7 +35,6 @@ import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
-import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.util.concurrent.CountDownLatch;
@@ -116,8 +115,12 @@ public class TestBlockingRpc {
   public void setUpRpcClient() throws Exception {
     retries = 1;
 
-    client = new BlockingRpcClient(DummyProtocol.class,
-        RpcUtils.getConnectAddress(server.getListenAddress()), retries);
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.getConnectAddress(server.getListenAddress()),
+              DummyProtocol.class, false);
+    client = new BlockingRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
     stub = client.getStub();
   }
 
@@ -238,7 +241,10 @@ public class TestBlockingRpc {
     });
     serverThread.start();
 
-    client = new BlockingRpcClient(DummyProtocol.class, address, retries);
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(address, DummyProtocol.class, false);
+    client = new BlockingRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
     stub = client.getStub();
 
     EchoMessage response = stub.echo(null, message);
@@ -247,24 +253,23 @@ public class TestBlockingRpc {
 
   @Test
   public void testConnectionFailed() throws Exception {
-    boolean expected = false;
     NettyClientBase client = null;
     
     try {
       int port = server.getListenAddress().getPort() + 1;
-      client = new BlockingRpcClient(DummyProtocol.class,
-          RpcUtils.getConnectAddress(new InetSocketAddress("127.0.0.1", port)), retries);
+      RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.getConnectAddress(new InetSocketAddress("127.0.0.1", port)),
+              DummyProtocol.class, false);
+      client = new BlockingRpcClient(rpcConnectionKey, retries);
+      assertFalse(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
       client.close();
-      fail("Connection should be failed.");
-    } catch (ConnectException ce) {
-      expected = true;
     } catch (Throwable ce){
       if (client != null) {
         client.close();
       }
       fail();
     }
-    assertTrue(expected);
   }
 
   @Test
@@ -329,8 +334,11 @@ public class TestBlockingRpc {
   @SetupRpcConnection(setupRpcClient=false)
   public void testUnresolvedAddress() throws Exception {
     String hostAndPort = RpcUtils.normalizeInetSocketAddress(server.getListenAddress());
-    client = new BlockingRpcClient(DummyProtocol.class,
-        RpcUtils.createUnresolved(hostAndPort), retries);
+    RpcConnectionPool.RpcConnectionKey rpcConnectionKey =
+          new RpcConnectionPool.RpcConnectionKey(
+              RpcUtils.createUnresolved(hostAndPort), DummyProtocol.class, false);
+    client = new BlockingRpcClient(rpcConnectionKey, retries);
+    assertTrue(client.acquire(RpcConnectionPool.DEFAULT_TIMEOUT));
     BlockingInterface stub = client.getStub();
 
     EchoMessage message = EchoMessage.newBuilder()


[11/13] tajo git commit: TAJO-1337: Implements common modules to handle RESTful API

Posted by ji...@apache.org.
TAJO-1337: Implements common modules to handle RESTful API

Closes #399


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

Branch: refs/heads/index_support
Commit: a9ae3cab69526294475a771014e9c0e49c80462b
Parents: 82d44af
Author: Jihun Kang <ji...@apache.org>
Authored: Wed Mar 18 23:57:06 2015 +0900
Committer: Jihun Kang <ji...@apache.org>
Committed: Wed Mar 18 23:57:06 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 tajo-catalog/tajo-catalog-client/pom.xml        |   2 +-
 .../tajo-catalog-drivers/tajo-hcatalog/pom.xml  |   2 +-
 tajo-catalog/tajo-catalog-server/pom.xml        |   2 +-
 tajo-cli/pom.xml                                |   2 +-
 tajo-client/pom.xml                             |   2 +-
 .../java/org/apache/tajo/conf/TajoConf.java     |   2 +
 tajo-core/pom.xml                               |   2 +-
 tajo-dist/pom.xml                               |   7 +-
 tajo-project/pom.xml                            |  31 ++
 tajo-pullserver/pom.xml                         |   2 +-
 tajo-rpc/pom.xml                                | 180 ++--------
 .../org/apache/tajo/rpc/AsyncRpcClient.java     | 227 ------------
 .../org/apache/tajo/rpc/AsyncRpcServer.java     | 148 --------
 .../org/apache/tajo/rpc/BlockingRpcClient.java  | 273 ---------------
 .../org/apache/tajo/rpc/BlockingRpcServer.java  | 147 --------
 .../java/org/apache/tajo/rpc/CallFuture.java    |  84 -----
 .../apache/tajo/rpc/DefaultRpcController.java   |  65 ----
 .../org/apache/tajo/rpc/NettyClientBase.java    | 221 ------------
 .../org/apache/tajo/rpc/NettyRpcController.java |  63 ----
 .../org/apache/tajo/rpc/NettyServerBase.java    | 205 -----------
 .../java/org/apache/tajo/rpc/NullCallback.java  |  38 --
 .../tajo/rpc/ProtoChannelInitializer.java       |  50 ---
 .../apache/tajo/rpc/RemoteCallException.java    |  69 ----
 .../org/apache/tajo/rpc/RemoteException.java    |  37 --
 .../tajo/rpc/RetriesExhaustedException.java     | 104 ------
 .../org/apache/tajo/rpc/RpcChannelFactory.java  | 182 ----------
 .../org/apache/tajo/rpc/RpcConnectionPool.java  | 190 ----------
 .../main/java/org/apache/tajo/rpc/RpcUtils.java | 122 -------
 .../org/apache/tajo/rpc/ServerCallable.java     | 162 ---------
 .../apache/tajo/rpc/TajoServiceException.java   |  58 ---
 tajo-rpc/src/main/proto/DummyProtos.proto       |  47 ---
 tajo-rpc/src/main/proto/RpcProtos.proto         |  32 --
 tajo-rpc/src/main/proto/TestProtocol.proto      |  31 --
 tajo-rpc/src/main/proto/TestProtos.proto        |  35 --
 tajo-rpc/src/test/java/log4j.properties         |  25 --
 .../java/org/apache/tajo/rpc/TestAsyncRpc.java  | 345 ------------------
 .../org/apache/tajo/rpc/TestBlockingRpc.java    | 349 -------------------
 .../rpc/test/impl/DummyProtocolAsyncImpl.java   |  86 -----
 .../test/impl/DummyProtocolBlockingImpl.java    |  83 -----
 tajo-rpc/tajo-rpc-common/pom.xml                | 216 ++++++++++++
 .../org/apache/tajo/rpc/NettyServerBase.java    | 243 +++++++++++++
 .../org/apache/tajo/rpc/RemoteException.java    |  37 ++
 .../tajo/rpc/RetriesExhaustedException.java     | 104 ++++++
 .../org/apache/tajo/rpc/RpcChannelFactory.java  | 182 ++++++++++
 .../org/apache/tajo/rpc/RpcEventListener.java   |  62 ++++
 .../main/java/org/apache/tajo/rpc/RpcUtils.java | 122 +++++++
 tajo-rpc/tajo-rpc-protobuf/pom.xml              | 274 +++++++++++++++
 .../org/apache/tajo/rpc/AsyncRpcClient.java     | 227 ++++++++++++
 .../org/apache/tajo/rpc/AsyncRpcServer.java     | 148 ++++++++
 .../org/apache/tajo/rpc/BlockingRpcClient.java  | 273 +++++++++++++++
 .../org/apache/tajo/rpc/BlockingRpcServer.java  | 147 ++++++++
 .../java/org/apache/tajo/rpc/CallFuture.java    |  84 +++++
 .../apache/tajo/rpc/DefaultRpcController.java   |  65 ++++
 .../org/apache/tajo/rpc/NettyClientBase.java    | 221 ++++++++++++
 .../org/apache/tajo/rpc/NettyRpcController.java |  63 ++++
 .../java/org/apache/tajo/rpc/NullCallback.java  |  38 ++
 .../tajo/rpc/ProtoChannelInitializer.java       |  50 +++
 .../apache/tajo/rpc/RemoteCallException.java    |  69 ++++
 .../org/apache/tajo/rpc/RemoteException.java    |  37 ++
 .../tajo/rpc/RetriesExhaustedException.java     | 104 ++++++
 .../org/apache/tajo/rpc/RpcConnectionPool.java  | 190 ++++++++++
 .../org/apache/tajo/rpc/ServerCallable.java     | 162 +++++++++
 .../apache/tajo/rpc/TajoServiceException.java   |  58 +++
 .../src/main/proto/DummyProtos.proto            |  47 +++
 .../src/main/proto/RpcProtos.proto              |  32 ++
 .../src/main/proto/TestProtocol.proto           |  31 ++
 .../src/main/proto/TestProtos.proto             |  35 ++
 .../src/test/java/log4j.properties              |  25 ++
 .../java/org/apache/tajo/rpc/TestAsyncRpc.java  | 345 ++++++++++++++++++
 .../org/apache/tajo/rpc/TestBlockingRpc.java    | 349 +++++++++++++++++++
 .../rpc/test/impl/DummyProtocolAsyncImpl.java   |  86 +++++
 .../test/impl/DummyProtocolBlockingImpl.java    |  83 +++++
 tajo-rpc/tajo-ws-rs/pom.xml                     | 218 ++++++++++++
 .../rs/netty/NettyRestChannelInitializer.java   |  50 +++
 .../ws/rs/netty/NettyRestHandlerContainer.java  | 319 +++++++++++++++++
 .../NettyRestHandlerContainerProvider.java      |  42 +++
 .../tajo/ws/rs/netty/NettyRestServer.java       |  67 ++++
 .../ws/rs/netty/NettyRestServerFactory.java     |  89 +++++
 .../ws/rs/netty/NettyRestServerListener.java    |  72 ++++
 .../tajo/ws/rs/netty/gson/GsonFeature.java      |  34 ++
 .../tajo/ws/rs/netty/gson/GsonReader.java       |  52 +++
 .../apache/tajo/ws/rs/netty/gson/GsonUtil.java  |  32 ++
 .../tajo/ws/rs/netty/gson/GsonWriter.java       |  59 ++++
 .../NettyRestHandlerContainerProviderTest.java  |  66 ++++
 .../tajo/ws/rs/netty/NettyRestServerTest.java   | 137 ++++++++
 .../ws/rs/netty/testapp1/TestApplication1.java  |  38 ++
 .../ws/rs/netty/testapp1/TestResource1.java     |  36 ++
 .../ws/rs/netty/testapp2/DirectoriesDao.java    |  39 +++
 .../rs/netty/testapp2/DirectoriesResource.java  |  85 +++++
 .../tajo/ws/rs/netty/testapp2/Directory.java    |  52 +++
 .../testapp2/FileManagementApplication.java     |  35 ++
 92 files changed, 5809 insertions(+), 3636 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index c3f2691..4875cab 100644
--- a/CHANGES
+++ b/CHANGES
@@ -76,6 +76,8 @@ Release 0.11.0 - unreleased
 
   SUB TASKS
 
+    TAJO-1337: Implements common modules to handle RESTful API. (jihun)
+
     TAJO-1329: Improve Schema class to support nested struct support.
     (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-catalog/tajo-catalog-client/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/pom.xml b/tajo-catalog/tajo-catalog-client/pom.xml
index 98b85a8..84e2aa3 100644
--- a/tajo-catalog/tajo-catalog-client/pom.xml
+++ b/tajo-catalog/tajo-catalog-client/pom.xml
@@ -135,7 +135,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
index fe8f34a..7c3efdd 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
@@ -109,7 +109,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-catalog/tajo-catalog-server/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/pom.xml b/tajo-catalog/tajo-catalog-server/pom.xml
index 501f9af..8efeecf 100644
--- a/tajo-catalog/tajo-catalog-server/pom.xml
+++ b/tajo-catalog/tajo-catalog-server/pom.xml
@@ -141,7 +141,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-cli/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-cli/pom.xml b/tajo-cli/pom.xml
index 684c298..e8360ad 100644
--- a/tajo-cli/pom.xml
+++ b/tajo-cli/pom.xml
@@ -140,7 +140,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
     </dependency>
     <dependency>
       <groupId>commons-cli</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-client/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-client/pom.xml b/tajo-client/pom.xml
index 692e1b5..e6be476 100644
--- a/tajo-client/pom.xml
+++ b/tajo-client/pom.xml
@@ -195,7 +195,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index 4ed8097..5b569d5 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -249,6 +249,8 @@ public class TajoConf extends Configuration {
         Runtime.getRuntime().availableProcessors() * 1),
     WORKER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM("tajo.worker.service.rpc.server.worker-thread-num",
         Runtime.getRuntime().availableProcessors() * 1),
+    REST_SERVICE_RPC_SERVER_WORKER_THREAD_NUM("tajo.rest.service.rpc.server.worker-thread-num",
+        Runtime.getRuntime().availableProcessors() * 1),
 
     // Task Configuration -----------------------------------------------------
     TASK_DEFAULT_MEMORY("tajo.task.memory-slot-mb.default", 512),

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-core/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml
index 38bddec..61a156b 100644
--- a/tajo-core/pom.xml
+++ b/tajo-core/pom.xml
@@ -277,7 +277,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index aed7b4b..da5f48f 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -60,7 +60,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-ws-rs</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index 9f1b1ab..37121e3 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -38,6 +38,7 @@
     <tajo.version>0.11.0-SNAPSHOT</tajo.version>
     <hbase.version>0.98.7-hadoop2</hbase.version>
     <netty.version>4.0.25.Final</netty.version>
+    <jersey.version>2.6</jersey.version>
     <tajo.root>${project.parent.relativePath}/..</tajo.root>
     <extra.source.path>src/main/hadoop-${hadoop.version}</extra.source.path>
   </properties>
@@ -789,6 +790,21 @@
       </dependency>
       <dependency>
         <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-rpc-common</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-rpc-protobuf</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-ws-rs</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
         <artifactId>tajo-algebra</artifactId>
         <version>${tajo.version}</version>
       </dependency>
@@ -1063,6 +1079,21 @@
         <artifactId>jcip-annotations</artifactId>
         <version>1.0-1</version>
       </dependency>
+      <dependency>
+        <groupId>org.glassfish.jersey.core</groupId>
+        <artifactId>jersey-common</artifactId>
+        <version>${jersey.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.glassfish.jersey.core</groupId>
+        <artifactId>jersey-server</artifactId>
+        <version>${jersey.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>javax.ws.rs</groupId>
+        <artifactId>javax.ws.rs-api</artifactId>
+        <version>2.0.1</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
   <profiles>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-pullserver/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-pullserver/pom.xml b/tajo-pullserver/pom.xml
index 944cf3d..ba6e6b7 100644
--- a/tajo-pullserver/pom.xml
+++ b/tajo-pullserver/pom.xml
@@ -56,7 +56,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-rpc</artifactId>
+      <artifactId>tajo-rpc-protobuf</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-rpc/pom.xml b/tajo-rpc/pom.xml
index 8c626b4..f069aca 100644
--- a/tajo-rpc/pom.xml
+++ b/tajo-rpc/pom.xml
@@ -24,165 +24,39 @@
     <groupId>org.apache.tajo</groupId>
     <relativePath>../tajo-project</relativePath>
   </parent>
-  <packaging>jar</packaging>
   <artifactId>tajo-rpc</artifactId>
-  <name>Tajo Rpc</name>
-  <description>RPC Server/Client Implementation based on Netty and Protocol Buffer</description>
+  <packaging>pom</packaging>
+  <name>Tajo RPC</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+  </properties>
+
+  <modules>
+    <module>tajo-rpc-common</module>
+    <module>tajo-rpc-protobuf</module>
+    <module>tajo-ws-rs</module>
+  </modules>
 
   <build>
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
-          <encoding>${project.build.sourceEncoding}</encoding>
-        </configuration>
-      </plugin>
-      <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>verify</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-          </execution>
-        </executions>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <version>2.4</version>
-        <configuration>
-        </configuration>
-        <executions>
-          <execution>
-            <id>create-jar</id>
-            <phase>prepare-package</phase>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>create-protobuf-generated-sources-directory</id>
-            <phase>initialize</phase>
-            <configuration>
-              <target>
-                <mkdir dir="target/generated-sources/proto" />
-              </target>
-            </configuration>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <version>1.2</version>
-        <executions>
-          <execution>
-            <id>generate-sources</id>
-            <phase>generate-sources</phase>
-            <configuration>
-              <executable>protoc</executable>
-              <arguments>
-                <argument>-Isrc/main/proto/</argument>
-                <argument>--java_out=target/generated-sources/proto</argument>
-                <argument>src/main/proto/DummyProtos.proto</argument>
-                <argument>src/main/proto/RpcProtos.proto</argument>
-                <argument>src/main/proto/TestProtos.proto</argument>
-                <argument>src/main/proto/TestProtocol.proto</argument>
-              </arguments>
-            </configuration>
-            <goals>
-              <goal>exec</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>build-helper-maven-plugin</artifactId>
-        <version>1.5</version>
-        <executions>
-          <execution>
-            <id>add-source</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>add-source</goal>
-            </goals>
-            <configuration>
-              <sources>
-                <source>target/generated-sources/proto</source>
-              </sources>
-            </configuration>
-          </execution>
-        </executions>
+        <artifactId>maven-surefire-report-plugin</artifactId>
       </plugin>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-report-plugin</artifactId>
-        <version>2.15</version>
+        <artifactId>maven-deploy-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
       </plugin>
     </plugins>
   </build>
 
-  <dependencies>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-transport</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-codec</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-handler</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-lang</groupId>
-      <artifactId>commons-lang</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-  </properties>
-  <repositories>
-    <repository>
-      <id>repository.jboss.org</id>
-      <url>https://repository.jboss.org/nexus/content/repositories/releases/
-			</url>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </repository>
-  </repositories>
 
   <profiles>
     <profile>
@@ -216,6 +90,9 @@
       <id>dist</id>
       <activation>
         <activeByDefault>false</activeByDefault>
+        <property>
+          <name>tar|rpm|deb</name>
+        </property>
       </activation>
       <build>
         <plugins>
@@ -225,7 +102,7 @@
             <executions>
               <execution>
                 <id>dist</id>
-                <phase>package</phase>
+                <phase>prepare-package</phase>
                 <goals>
                   <goal>run</goal>
                 </goals>
@@ -248,12 +125,15 @@
                       echo
                       echo "Current directory `pwd`"
                       echo
-                      run rm -rf ${project.artifactId}-${project.version}
-                      run mkdir ${project.artifactId}-${project.version}
-                      run cd ${project.artifactId}-${project.version}
-                      run cp -r ${basedir}/target/${project.artifactId}-${project.version}*.jar .
+                      run rm -rf tajo-rpc-${project.version}
+                      run mkdir tajo-rpc-${project.version}
+                      run cd tajo-rpc-${project.version}
+                      run cp -r ${basedir}/tajo-rpc-common/target/tajo-rpc-common-${project.version}*.jar .
+                      run cp -r ${basedir}/tajo-rpc-protobuf/target/tajo-rpc-protobuf-${project.version}*.jar .
+                      run cp -r ${basedir}/tajo-ws-rs/target/tajo-ws-rs-${project.version}*.jar .
+
                       echo
-                      echo "Tajo Rpc dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo "Tajo RPC dist layout available at: ${project.build.directory}/tajo-rpc-${project.version}"
                       echo
                     </echo>
                     <exec executable="sh" dir="${project.build.directory}" failonerror="true">
@@ -274,9 +154,9 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-report-plugin</artifactId>
-        <version>2.15</version>
       </plugin>
     </plugins>
   </reporting>
 
 </project>
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
deleted file mode 100644
index 3d856ce..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java
+++ /dev/null
@@ -1,227 +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.rpc;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.*;
-
-import io.netty.channel.*;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
-import org.apache.tajo.rpc.RpcProtos.RpcRequest;
-import org.apache.tajo.rpc.RpcProtos.RpcResponse;
-
-import io.netty.util.ReferenceCountUtil;
-import io.netty.util.concurrent.GenericFutureListener;
-
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-public class AsyncRpcClient extends NettyClientBase {
-  private static final Log LOG = LogFactory.getLog(AsyncRpcClient.class);
-
-  private final ConcurrentMap<Integer, ResponseCallback> requests =
-      new ConcurrentHashMap<Integer, ResponseCallback>();
-
-  private final Method stubMethod;
-  private final ProxyRpcChannel rpcChannel;
-  private final ClientChannelInboundHandler inboundHandler;
-
-  /**
-   * Intentionally make this method package-private, avoiding user directly
-   * new an instance through this constructor.
-   */
-  AsyncRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
-      throws ClassNotFoundException, NoSuchMethodException {
-    super(rpcConnectionKey, retries);
-    stubMethod = getServiceClass().getMethod("newStub", RpcChannel.class);
-    rpcChannel = new ProxyRpcChannel();
-    inboundHandler = new ClientChannelInboundHandler();
-    init(new ProtoChannelInitializer(inboundHandler, RpcResponse.getDefaultInstance()));
-  }
-
-  @Override
-  public <T> T getStub() {
-    return getStub(stubMethod, rpcChannel);
-  }
-
-  protected void sendExceptions(String message) {
-    for(Map.Entry<Integer, ResponseCallback> callbackEntry: requests.entrySet()) {
-      ResponseCallback callback = callbackEntry.getValue();
-      Integer id = callbackEntry.getKey();
-
-      RpcResponse.Builder responseBuilder = RpcResponse.newBuilder()
-          .setErrorMessage(message)
-          .setId(id);
-
-      callback.run(responseBuilder.build());
-    }
-  }
-
-  @Override
-  public void close() {
-    sendExceptions("AsyncRpcClient terminates all the connections");
-
-    super.close();
-  }
-
-  private class ProxyRpcChannel implements RpcChannel {
-
-    public void callMethod(final MethodDescriptor method,
-                           final RpcController controller,
-                           final Message param,
-                           final Message responseType,
-                           RpcCallback<Message> done) {
-
-      int nextSeqId = sequence.getAndIncrement();
-
-      Message rpcRequest = buildRequest(nextSeqId, method, param);
-
-      inboundHandler.registerCallback(nextSeqId,
-          new ResponseCallback(controller, responseType, done));
-
-      ChannelPromise channelPromise = getChannel().newPromise();
-      channelPromise.addListener(new GenericFutureListener<ChannelFuture>() {
-
-        @Override
-        public void operationComplete(ChannelFuture future) throws Exception {
-          if (!future.isSuccess()) {
-            inboundHandler.exceptionCaught(null, new ServiceException(future.cause()));
-          }
-        }
-      });
-      getChannel().writeAndFlush(rpcRequest, channelPromise);
-    }
-
-    private Message buildRequest(int seqId,
-                                 MethodDescriptor method,
-                                 Message param) {
-
-      RpcRequest.Builder requestBuilder = RpcRequest.newBuilder()
-          .setId(seqId)
-          .setMethodName(method.getName());
-
-      if (param != null) {
-          requestBuilder.setRequestMessage(param.toByteString());
-      }
-
-      return requestBuilder.build();
-    }
-  }
-
-  private class ResponseCallback implements RpcCallback<RpcResponse> {
-    private final RpcController controller;
-    private final Message responsePrototype;
-    private final RpcCallback<Message> callback;
-
-    public ResponseCallback(RpcController controller,
-                            Message responsePrototype,
-                            RpcCallback<Message> callback) {
-      this.controller = controller;
-      this.responsePrototype = responsePrototype;
-      this.callback = callback;
-    }
-
-    @Override
-    public void run(RpcResponse rpcResponse) {
-      // if hasErrorMessage is true, it means rpc-level errors.
-      // it does not call the callback function\
-      if (rpcResponse.hasErrorMessage()) {
-        if (controller != null) {
-          this.controller.setFailed(rpcResponse.getErrorMessage());
-        }
-        callback.run(null);
-      } else { // if rpc call succeed
-        try {
-          Message responseMessage;
-          if (!rpcResponse.hasResponseMessage()) {
-            responseMessage = null;
-          } else {
-            responseMessage = responsePrototype.newBuilderForType().mergeFrom(
-                rpcResponse.getResponseMessage()).build();
-          }
-
-          callback.run(responseMessage);
-
-        } catch (InvalidProtocolBufferException e) {
-          throw new RemoteException(getErrorMessage(""), e);
-        }
-      }
-    }
-  }
-
-  private String getErrorMessage(String message) {
-    return "Exception [" + protocol.getCanonicalName() +
-        "(" + RpcUtils.normalizeInetSocketAddress((InetSocketAddress)
-        getChannel().remoteAddress()) + ")]: " + message;
-  }
-
-  @ChannelHandler.Sharable
-  private class ClientChannelInboundHandler extends ChannelInboundHandlerAdapter {
-
-    void registerCallback(int seqId, ResponseCallback callback) {
-
-      if (requests.putIfAbsent(seqId, callback) != null) {
-        throw new RemoteException(
-            getErrorMessage("Duplicate Sequence Id "+ seqId));
-      }
-    }
-
-    @Override
-    public void channelRead(ChannelHandlerContext ctx, Object msg)
-        throws Exception {
-      if (msg instanceof RpcResponse) {
-        try {
-          RpcResponse response = (RpcResponse) msg;
-          ResponseCallback callback = requests.remove(response.getId());
-
-          if (callback == null) {
-            LOG.warn("Dangling rpc call");
-          } else {
-            callback.run(response);
-          }
-        } finally {
-          ReferenceCountUtil.release(msg);
-        }
-      }
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
-        throws Exception {
-      LOG.error(getRemoteAddress() + "," + protocol + "," + cause.getMessage(), cause);
-
-      sendExceptions(cause.getMessage());
-      
-      if(LOG.isDebugEnabled()) {
-        LOG.error(cause.getMessage(), cause);
-      } else {
-        LOG.error("RPC Exception:" + cause.getMessage());
-      }
-      
-      if (ctx != null && ctx.channel().isActive()) {
-        ctx.channel().close();
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java
deleted file mode 100644
index 3b5a747..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/AsyncRpcServer.java
+++ /dev/null
@@ -1,148 +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.rpc;
-
-import com.google.protobuf.*;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-
-import io.netty.channel.*;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.RpcProtos.RpcRequest;
-import org.apache.tajo.rpc.RpcProtos.RpcResponse;
-
-import io.netty.util.ReferenceCountUtil;
-
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-
-public class AsyncRpcServer extends NettyServerBase {
-  private static final Log LOG = LogFactory.getLog(AsyncRpcServer.class);
-
-  private final Service service;
-  private final ChannelInitializer<Channel> initializer;
-
-  public AsyncRpcServer(final Class<?> protocol,
-                        final Object instance,
-                        final InetSocketAddress bindAddress,
-                        final int workerNum)
-      throws Exception {
-    super(protocol.getSimpleName(), bindAddress);
-
-    String serviceClassName = protocol.getName() + "$" +
-        protocol.getSimpleName() + "Service";
-    Class<?> serviceClass = Class.forName(serviceClassName);
-    Class<?> interfaceClass = Class.forName(serviceClassName + "$Interface");
-    Method method = serviceClass.getMethod("newReflectiveService", interfaceClass);
-    this.service = (Service) method.invoke(null, instance);
-
-    this.initializer = new ProtoChannelInitializer(new ServerHandler(), RpcRequest.getDefaultInstance());
-    super.init(this.initializer, workerNum);
-  }
-
-  @ChannelHandler.Sharable
-  private class ServerHandler extends ChannelInboundHandlerAdapter {
-
-    @Override
-    public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
-      accepted.add(ctx.channel());
-      if(LOG.isDebugEnabled()){
-        LOG.debug(String.format(serviceName + " accepted number of connections (%d)", accepted.size()));
-      }
-      super.channelRegistered(ctx);
-    }
-
-    @Override
-    public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
-      accepted.remove(ctx.channel());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(serviceName + " closes a connection. The number of current connections are " + accepted.size());
-      }
-      super.channelUnregistered(ctx);
-    }
-
-    @Override
-    public void channelRead(final ChannelHandlerContext ctx, Object msg)
-        throws Exception {
-      if (msg instanceof RpcRequest) {
-        try {
-          final RpcRequest request = (RpcRequest) msg;
-
-          String methodName = request.getMethodName();
-          MethodDescriptor methodDescriptor = service.getDescriptorForType().findMethodByName(methodName);
-
-          if (methodDescriptor == null) {
-            throw new RemoteCallException(request.getId(), new NoSuchMethodException(methodName));
-          }
-
-          Message paramProto = null;
-          if (request.hasRequestMessage()) {
-            try {
-              paramProto = service.getRequestPrototype(methodDescriptor).newBuilderForType()
-                  .mergeFrom(request.getRequestMessage()).build();
-            } catch (Throwable t) {
-              throw new RemoteCallException(request.getId(), methodDescriptor, t);
-            }
-          }
-
-          final RpcController controller = new NettyRpcController();
-
-          RpcCallback<Message> callback = !request.hasId() ? null : new RpcCallback<Message>() {
-
-            public void run(Message returnValue) {
-
-              RpcResponse.Builder builder = RpcResponse.newBuilder().setId(request.getId());
-
-              if (returnValue != null) {
-                builder.setResponseMessage(returnValue.toByteString());
-              }
-
-              if (controller.failed()) {
-                builder.setErrorMessage(controller.errorText());
-              }
-
-              ctx.writeAndFlush(builder.build());
-            }
-          };
-
-          service.callMethod(methodDescriptor, controller, paramProto, callback);
-
-        } finally {
-          ReferenceCountUtil.release(msg);
-        }
-      }
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
-        throws Exception{
-      if (cause instanceof RemoteCallException) {
-        RemoteCallException callException = (RemoteCallException) cause;
-        ctx.writeAndFlush(callException.getResponse());
-      } else {
-        LOG.error(cause.getMessage());
-      }
-      
-      if (ctx != null && ctx.channel().isActive()) {
-        ctx.channel().close();
-      }
-    }
-    
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
deleted file mode 100644
index 6a90330..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java
+++ /dev/null
@@ -1,273 +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.rpc;
-
-import com.google.protobuf.*;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-
-import io.netty.channel.*;
-import io.netty.util.concurrent.*;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
-import org.apache.tajo.rpc.RpcProtos.RpcRequest;
-import org.apache.tajo.rpc.RpcProtos.RpcResponse;
-
-import io.netty.util.ReferenceCountUtil;
-
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.concurrent.*;
-import java.util.concurrent.Future;
-
-public class BlockingRpcClient extends NettyClientBase {
-  private static final Log LOG = LogFactory.getLog(RpcProtos.class);
-
-  private final Map<Integer, ProtoCallFuture> requests =
-      new ConcurrentHashMap<Integer, ProtoCallFuture>();
-
-  private final Method stubMethod;
-  private final ProxyRpcChannel rpcChannel;
-  private final ChannelInboundHandlerAdapter inboundHandler;
-
-  /**
-   * Intentionally make this method package-private, avoiding user directly
-   * new an instance through this constructor.
-   */
-  BlockingRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
-      throws ClassNotFoundException, NoSuchMethodException {
-    super(rpcConnectionKey, retries);
-    stubMethod = getServiceClass().getMethod("newBlockingStub", BlockingRpcChannel.class);
-    rpcChannel = new ProxyRpcChannel();
-    inboundHandler = new ClientChannelInboundHandler();
-    init(new ProtoChannelInitializer(inboundHandler, RpcResponse.getDefaultInstance()));
-  }
-
-  @Override
-  public <T> T getStub() {
-    return getStub(stubMethod, rpcChannel);
-  }
-
-  @Override
-  public void close() {
-    for(ProtoCallFuture callback: requests.values()) {
-      callback.setFailed("BlockingRpcClient terminates all the connections",
-          new ServiceException("BlockingRpcClient terminates all the connections"));
-    }
-
-    super.close();
-  }
-
-  private class ProxyRpcChannel implements BlockingRpcChannel {
-
-    @Override
-    public Message callBlockingMethod(final MethodDescriptor method,
-                                      final RpcController controller,
-                                      final Message param,
-                                      final Message responsePrototype)
-        throws TajoServiceException {
-
-      int nextSeqId = sequence.getAndIncrement();
-
-      Message rpcRequest = buildRequest(nextSeqId, method, param);
-
-      ProtoCallFuture callFuture =
-          new ProtoCallFuture(controller, responsePrototype);
-      requests.put(nextSeqId, callFuture);
-
-      ChannelPromise channelPromise = getChannel().newPromise();
-      channelPromise.addListener(new GenericFutureListener<ChannelFuture>() {
-        @Override
-        public void operationComplete(ChannelFuture future) throws Exception {
-          if (!future.isSuccess()) {
-            inboundHandler.exceptionCaught(null, new ServiceException(future.cause()));
-          }
-        }
-      });
-      getChannel().writeAndFlush(rpcRequest, channelPromise);
-
-      try {
-        return callFuture.get(60, TimeUnit.SECONDS);
-      } catch (Throwable t) {
-        if (t instanceof ExecutionException) {
-          Throwable cause = t.getCause();
-          if (cause != null && cause instanceof TajoServiceException) {
-            throw (TajoServiceException)cause;
-          }
-        }
-        throw new TajoServiceException(t.getMessage());
-      }
-    }
-
-    private Message buildRequest(int seqId,
-                                 MethodDescriptor method,
-                                 Message param) {
-      RpcRequest.Builder requestBuilder = RpcRequest.newBuilder()
-          .setId(seqId)
-          .setMethodName(method.getName());
-
-      if (param != null) {
-        requestBuilder.setRequestMessage(param.toByteString());
-      }
-
-      return requestBuilder.build();
-    }
-  }
-
-  private String getErrorMessage(String message) {
-    if(getChannel() != null) {
-      return protocol.getName() +
-          "(" + RpcUtils.normalizeInetSocketAddress((InetSocketAddress)
-          getChannel().remoteAddress()) + "): " + message;
-    } else {
-      return "Exception " + message;
-    }
-  }
-
-  private TajoServiceException makeTajoServiceException(RpcResponse response, Throwable cause) {
-    if(getChannel() != null) {
-      return new TajoServiceException(response.getErrorMessage(), cause, protocol.getName(),
-          RpcUtils.normalizeInetSocketAddress((InetSocketAddress)getChannel().remoteAddress()));
-    } else {
-      return new TajoServiceException(response.getErrorMessage());
-    }
-  }
-
-  @ChannelHandler.Sharable
-  private class ClientChannelInboundHandler extends ChannelInboundHandlerAdapter {
-
-    @Override
-    public void channelRead(ChannelHandlerContext ctx, Object msg)
-        throws Exception {
-
-      if (msg instanceof RpcResponse) {
-        try {
-          RpcResponse rpcResponse = (RpcResponse) msg;
-          ProtoCallFuture callback = requests.remove(rpcResponse.getId());
-
-          if (callback == null) {
-            LOG.warn("Dangling rpc call");
-          } else {
-            if (rpcResponse.hasErrorMessage()) {
-              callback.setFailed(rpcResponse.getErrorMessage(),
-                  makeTajoServiceException(rpcResponse, new ServiceException(rpcResponse.getErrorTrace())));
-              throw new RemoteException(getErrorMessage(rpcResponse.getErrorMessage()));
-            } else {
-              Message responseMessage;
-
-              if (!rpcResponse.hasResponseMessage()) {
-                responseMessage = null;
-              } else {
-                responseMessage = callback.returnType.newBuilderForType().mergeFrom(rpcResponse.getResponseMessage())
-                    .build();
-              }
-
-              callback.setResponse(responseMessage);
-            }
-          }
-        } finally {
-          ReferenceCountUtil.release(msg);
-        }
-      }
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
-        throws Exception {
-      for(ProtoCallFuture callback: requests.values()) {
-        callback.setFailed(cause.getMessage(), cause);
-      }
-      
-      if(LOG.isDebugEnabled()) {
-        LOG.error("" + cause.getMessage(), cause);
-      } else {
-        LOG.error("RPC Exception:" + cause.getMessage());
-      }
-      if (ctx != null && ctx.channel().isActive()) {
-        ctx.channel().close();
-      }
-    }
-  }
-
- static class ProtoCallFuture implements Future<Message> {
-    private Semaphore sem = new Semaphore(0);
-    private Message response = null;
-    private Message returnType;
-
-    private RpcController controller;
-
-    private ExecutionException ee;
-
-    public ProtoCallFuture(RpcController controller, Message message) {
-      this.controller = controller;
-      this.returnType = message;
-    }
-
-    @Override
-    public boolean cancel(boolean arg0) {
-      return false;
-    }
-
-    @Override
-    public Message get() throws InterruptedException, ExecutionException {
-      sem.acquire();
-      if(ee != null) {
-        throw ee;
-      }
-      return response;
-    }
-
-    @Override
-    public Message get(long timeout, TimeUnit unit)
-        throws InterruptedException, ExecutionException, TimeoutException {
-      if(sem.tryAcquire(timeout, unit)) {
-        if (ee != null) {
-          throw ee;
-        }
-        return response;
-      } else {
-        throw new TimeoutException();
-      }
-    }
-
-    @Override
-    public boolean isCancelled() {
-      return false;
-    }
-
-    @Override
-    public boolean isDone() {
-      return sem.availablePermits() > 0;
-    }
-
-    public void setResponse(Message response) {
-      this.response = response;
-      sem.release();
-    }
-
-    public void setFailed(String errorText, Throwable t) {
-      if(controller != null) {
-        this.controller.setFailed(errorText);
-      }
-      ee = new ExecutionException(errorText, t);
-      sem.release();
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java
deleted file mode 100644
index 0ce359f..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/BlockingRpcServer.java
+++ /dev/null
@@ -1,147 +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.rpc;
-
-import com.google.protobuf.BlockingService;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcController;
-
-import io.netty.channel.*;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.RpcProtos.RpcRequest;
-import org.apache.tajo.rpc.RpcProtos.RpcResponse;
-
-import io.netty.util.ReferenceCountUtil;
-
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-
-public class BlockingRpcServer extends NettyServerBase {
-  private static Log LOG = LogFactory.getLog(BlockingRpcServer.class);
-  private final BlockingService service;
-  private final ChannelInitializer<Channel> initializer;
-
-  public BlockingRpcServer(final Class<?> protocol,
-                           final Object instance,
-                           final InetSocketAddress bindAddress,
-                           final int workerNum)
-      throws Exception {
-
-    super(protocol.getSimpleName(), bindAddress);
-
-    String serviceClassName = protocol.getName() + "$" +
-        protocol.getSimpleName() + "Service";
-    Class<?> serviceClass = Class.forName(serviceClassName);
-    Class<?> interfaceClass = Class.forName(serviceClassName +
-        "$BlockingInterface");
-    Method method = serviceClass.getMethod(
-        "newReflectiveBlockingService", interfaceClass);
-
-    this.service = (BlockingService) method.invoke(null, instance);
-    this.initializer = new ProtoChannelInitializer(new ServerHandler(), RpcRequest.getDefaultInstance());
-
-    super.init(this.initializer, workerNum);
-  }
-
-  @ChannelHandler.Sharable
-  private class ServerHandler extends ChannelInboundHandlerAdapter {
-
-    @Override
-    public void channelRegistered(ChannelHandlerContext ctx) throws Exception {
-      accepted.add(ctx.channel());
-      if(LOG.isDebugEnabled()){
-        LOG.debug(String.format(serviceName + " accepted number of connections (%d)", accepted.size()));
-      }
-      super.channelRegistered(ctx);
-    }
-
-    @Override
-    public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
-      accepted.remove(ctx.channel());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(serviceName + " closes a connection. The number of current connections are " + accepted.size());
-      }
-      super.channelUnregistered(ctx);
-    }
-
-    @Override
-    public void channelRead(ChannelHandlerContext ctx, Object msg)
-        throws Exception {
-
-      if (msg instanceof RpcRequest) {
-        try {
-          final RpcRequest request = (RpcRequest) msg;
-
-          String methodName = request.getMethodName();
-          MethodDescriptor methodDescriptor = service.getDescriptorForType().findMethodByName(methodName);
-
-          if (methodDescriptor == null) {
-            throw new RemoteCallException(request.getId(), new NoSuchMethodException(methodName));
-          }
-          Message paramProto = null;
-          if (request.hasRequestMessage()) {
-            try {
-              paramProto = service.getRequestPrototype(methodDescriptor).newBuilderForType()
-                  .mergeFrom(request.getRequestMessage()).build();
-
-            } catch (Throwable t) {
-              throw new RemoteCallException(request.getId(), methodDescriptor, t);
-            }
-          }
-          Message returnValue;
-          RpcController controller = new NettyRpcController();
-
-          try {
-            returnValue = service.callBlockingMethod(methodDescriptor, controller, paramProto);
-          } catch (Throwable t) {
-            throw new RemoteCallException(request.getId(), methodDescriptor, t);
-          }
-
-          RpcResponse.Builder builder = RpcResponse.newBuilder().setId(request.getId());
-
-          if (returnValue != null) {
-            builder.setResponseMessage(returnValue.toByteString());
-          }
-
-          if (controller.failed()) {
-            builder.setErrorMessage(controller.errorText());
-          }
-          ctx.writeAndFlush(builder.build());
-        } finally {
-          ReferenceCountUtil.release(msg);
-        }
-      }
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-      if (cause instanceof RemoteCallException) {
-        RemoteCallException callException = (RemoteCallException) cause;
-        ctx.writeAndFlush(callException.getResponse());
-      }
-      
-      if (ctx != null && ctx.channel().isActive()) {
-        ctx.channel().close();
-      }
-    }
-    
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/CallFuture.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/CallFuture.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/CallFuture.java
deleted file mode 100644
index c4c3256..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/CallFuture.java
+++ /dev/null
@@ -1,84 +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.rpc;
-
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-
-import java.util.concurrent.Future;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-public class CallFuture<T> implements RpcCallback<T>, Future<T> {
-
-  private final Semaphore sem = new Semaphore(0);
-  private boolean done = false;
-  private T response;
-  private RpcController controller;
-
-  public CallFuture() {
-    controller = new DefaultRpcController();
-  }
-
-  public RpcController getController() {
-    return controller;
-  }
-
-  @Override
-  public void run(T t) {
-    this.response = t;
-    done = true;
-    sem.release();
-  }
-
-  @Override
-  public boolean cancel(boolean mayInterruptIfRunning) {
-    controller.startCancel();
-    sem.release();
-    return controller.isCanceled();
-  }
-
-  @Override
-  public boolean isCancelled() {
-    return controller.isCanceled();
-  }
-
-  @Override
-  public boolean isDone() {
-    return done;
-  }
-
-  @Override
-  public T get() throws InterruptedException {
-    sem.acquire();
-
-    return response;
-  }
-
-  @Override
-  public T get(long timeout, TimeUnit unit)
-      throws InterruptedException, TimeoutException {
-    if (sem.tryAcquire(timeout, unit)) {
-      return response;
-    } else {
-      throw new TimeoutException();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java
deleted file mode 100644
index 4ba19a5..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/DefaultRpcController.java
+++ /dev/null
@@ -1,65 +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.rpc;
-
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-
-public class DefaultRpcController implements RpcController {
-  private String errorText;
-  private boolean error;
-  private boolean canceled;
-
-  @Override
-  public void reset() {
-    errorText = "";
-    error = false;
-    canceled = false;
-  }
-
-  @Override
-  public boolean failed() {
-    return error;
-  }
-
-  @Override
-  public String errorText() {
-    return errorText;
-  }
-
-  @Override
-  public void startCancel() {
-    this.canceled = true;
-  }
-
-  @Override
-  public void setFailed(String s) {
-    this.errorText = s;
-    this.error = true;
-  }
-
-  @Override
-  public boolean isCanceled() {
-    return canceled;
-  }
-
-  @Override
-  public void notifyOnCancel(RpcCallback<Object> objectRpcCallback) {
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
deleted file mode 100644
index 72278f2..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyClientBase.java
+++ /dev/null
@@ -1,221 +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.rpc;
-
-import io.netty.channel.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.rpc.RpcConnectionPool.RpcConnectionKey;
-
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.util.concurrent.GenericFutureListener;
-
-import java.io.Closeable;
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public abstract class NettyClientBase implements Closeable {
-  private static final Log LOG = LogFactory.getLog(NettyClientBase.class);
-  private static final int CONNECTION_TIMEOUT = 60000;  // 60 sec
-  private static final long PAUSE = 1000; // 1 sec
-
-  private final int numRetries;
-
-  private Bootstrap bootstrap;
-  private volatile ChannelFuture channelFuture;
-
-  protected final Class<?> protocol;
-  protected final AtomicInteger sequence = new AtomicInteger(0);
-
-  private final RpcConnectionKey key;
-  private final AtomicInteger counter = new AtomicInteger(0);   // reference counter
-
-  public NettyClientBase(RpcConnectionKey rpcConnectionKey, int numRetries)
-      throws ClassNotFoundException, NoSuchMethodException {
-    this.key = rpcConnectionKey;
-    this.protocol = rpcConnectionKey.protocolClass;
-    this.numRetries = numRetries;
-  }
-
-  // should be called from sub class
-  protected void init(ChannelInitializer<Channel> initializer) {
-    this.bootstrap = new Bootstrap();
-    this.bootstrap
-      .channel(NioSocketChannel.class)
-      .handler(initializer)
-      .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
-      .option(ChannelOption.SO_REUSEADDR, true)
-      .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, CONNECTION_TIMEOUT)
-      .option(ChannelOption.SO_RCVBUF, 1048576 * 10)
-      .option(ChannelOption.TCP_NODELAY, true);
-  }
-
-  public RpcConnectionPool.RpcConnectionKey getKey() {
-    return key;
-  }
-
-  protected final Class<?> getServiceClass() throws ClassNotFoundException {
-    String serviceClassName = protocol.getName() + "$" + protocol.getSimpleName() + "Service";
-    return Class.forName(serviceClassName);
-  }
-
-  @SuppressWarnings("unchecked")
-  protected final <T> T getStub(Method stubMethod, Object rpcChannel) {
-    try {
-      return (T) stubMethod.invoke(null, rpcChannel);
-    } catch (Exception e) {
-      throw new RemoteException(e.getMessage(), e);
-    }
-  }
-
-  public abstract <T> T getStub();
-
-  public boolean acquire(long timeout) {
-    if (!checkConnection(timeout)) {
-      return false;
-    }
-    counter.incrementAndGet();
-    return true;
-  }
-
-  public boolean release() {
-    return counter.decrementAndGet() == 0;
-  }
-
-  private boolean checkConnection(long timeout) {
-    if (isConnected()) {
-      return true;
-    }
-
-    InetSocketAddress addr = key.addr;
-    if (addr.isUnresolved()) {
-      addr = RpcUtils.createSocketAddr(addr.getHostName(), addr.getPort());
-    }
-
-    return handleConnectionInternally(addr, timeout);
-  }
-
-  private void connectUsingNetty(InetSocketAddress address, GenericFutureListener<ChannelFuture> listener) {
-    LOG.warn("Try to connect : " + address);
-    this.channelFuture = bootstrap.clone().group(RpcChannelFactory.getSharedClientEventloopGroup())
-            .connect(address)
-            .addListener(listener);
-  }
-
-  // first attendant kicks connection
-  private final RpcUtils.Scrutineer<CountDownLatch> connect = new RpcUtils.Scrutineer<CountDownLatch>();
-
-  private boolean handleConnectionInternally(final InetSocketAddress addr, long timeout) {
-    final CountDownLatch ticket = new CountDownLatch(1);
-    final CountDownLatch granted = connect.check(ticket);
-
-    // basically, it's double checked lock
-    if (ticket == granted && isConnected()) {
-      granted.countDown();
-      return true;
-    }
-
-    if (ticket == granted) {
-      connectUsingNetty(addr, new RetryConnectionListener(addr, granted));
-    }
-
-    try {
-      granted.await(timeout, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      // ignore
-    }
-
-    boolean success = channelFuture.isSuccess();
-
-    if (granted.getCount() == 0) {
-      connect.clear(granted);
-    }
-
-    return success;
-  }
-
-  class RetryConnectionListener implements GenericFutureListener<ChannelFuture> {
-    private final AtomicInteger retryCount = new AtomicInteger();
-    private final InetSocketAddress address;
-    private final CountDownLatch latch;
-
-    RetryConnectionListener(InetSocketAddress address, CountDownLatch latch) {
-      this.address = address;
-      this.latch = latch;
-    }
-
-    @Override
-    public void operationComplete(ChannelFuture channelFuture) throws Exception {
-      if (!channelFuture.isSuccess()) {
-        channelFuture.channel().close();
-
-        if (numRetries > retryCount.getAndIncrement()) {
-          final GenericFutureListener<ChannelFuture> currentListener = this;
-
-          RpcChannelFactory.getSharedClientEventloopGroup().schedule(new Runnable() {
-            @Override
-            public void run() {
-              connectUsingNetty(address, currentListener);
-            }
-          }, PAUSE, TimeUnit.MILLISECONDS);
-
-          LOG.debug("Connecting to " + address + " has been failed. Retrying to connect.");
-        }
-        else {
-          latch.countDown();
-
-          LOG.error("Max retry count has been exceeded. attempts=" + numRetries);
-        }
-      }
-      else {
-        latch.countDown();
-      }
-    }
-  }
-
-  public Channel getChannel() {
-    return channelFuture == null ? null : channelFuture.channel();
-  }
-
-  public boolean isConnected() {
-    Channel channel = getChannel();
-    return channel != null && channel.isOpen() && channel.isActive();
-  }
-
-  public SocketAddress getRemoteAddress() {
-    Channel channel = getChannel();
-    return channel == null ? null : channel.remoteAddress();
-  }
-
-  @Override
-  public void close() {
-    Channel channel = getChannel();
-    if (channel != null && channel.isOpen()) {
-      LOG.debug("Proxy will be disconnected from remote " + channel.remoteAddress());
-      channel.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyRpcController.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyRpcController.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyRpcController.java
deleted file mode 100644
index b7f4537..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyRpcController.java
+++ /dev/null
@@ -1,63 +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.rpc;
-
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-
-public class NettyRpcController implements RpcController {
-  private String errorText;
-
-  @Override
-  public void reset() {
-    errorText = null;
-  }
-
-  @Override
-  public boolean failed() {
-    return errorText != null;
-  }
-
-  @Override
-  public String errorText() {
-    return errorText;
-  }
-
-  @Override
-  public void startCancel() {
-    // TODO - to be implemented
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setFailed(String s) {
-    errorText = s;
-  }
-
-  @Override
-  public boolean isCanceled() {
-    // TODO - to be implemented
-    return false;
-  }
-
-  @Override
-  public void notifyOnCancel(RpcCallback<Object> objectRpcCallback) {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
deleted file mode 100644
index 024108b..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
+++ /dev/null
@@ -1,205 +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.rpc;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.group.ChannelGroup;
-import io.netty.channel.group.DefaultChannelGroup;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.util.concurrent.GlobalEventExecutor;
-
-import java.io.IOException;
-import java.net.DatagramSocket;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class NettyServerBase {
-  private static final Log LOG = LogFactory.getLog(NettyServerBase.class);
-  private static final String DEFAULT_PREFIX = "RpcServer_";
-  private static final AtomicInteger sequenceId = new AtomicInteger(0);
-
-  protected String serviceName;
-  protected InetSocketAddress serverAddr;
-  protected InetSocketAddress bindAddress;
-  protected ChannelInitializer<Channel> initializer;
-  protected ServerBootstrap bootstrap;
-  protected ChannelFuture channelFuture;
-  protected ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
-
-  private InetSocketAddress initIsa;
-
-  public NettyServerBase(InetSocketAddress address) {
-    this.initIsa = address;
-  }
-
-  public NettyServerBase(String serviceName, InetSocketAddress addr) {
-    this.serviceName = serviceName;
-    this.initIsa = addr;
-  }
-
-  public void setName(String name) {
-    this.serviceName = name;
-  }
-
-  public void init(ChannelInitializer<Channel> initializer, int workerNum) {
-    bootstrap = RpcChannelFactory.createServerChannelFactory(serviceName, workerNum);
-
-    this.initializer = initializer;
-    bootstrap
-      .channel(NioServerSocketChannel.class)
-      .childHandler(initializer)
-      .option(ChannelOption.SO_REUSEADDR, true)
-      .option(ChannelOption.TCP_NODELAY, true)
-      .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
-      .childOption(ChannelOption.TCP_NODELAY, true)
-      .childOption(ChannelOption.CONNECT_TIMEOUT_MILLIS, 10000)
-      .childOption(ChannelOption.SO_RCVBUF, 1048576 * 10);
-  }
-
-  public InetSocketAddress getListenAddress() {
-    return this.bindAddress;
-  }
-
-  public void start() {
-    if (serviceName == null) {
-      this.serviceName = getNextDefaultServiceName();
-    }
-
-    if (initIsa.getPort() == 0) {
-      try {
-        int port = getUnusedPort();
-        serverAddr = new InetSocketAddress(initIsa.getHostName(), port);
-      } catch (IOException e) {
-        LOG.error(e, e);
-      }
-    } else {
-      serverAddr = initIsa;
-    }
-
-    this.channelFuture = bootstrap.clone().bind(serverAddr).syncUninterruptibly();
-    this.bindAddress = (InetSocketAddress) channelFuture.channel().localAddress();
-
-    LOG.info("Rpc (" + serviceName + ") listens on " + this.bindAddress);
-  }
-
-  public Channel getChannel() {
-    return this.channelFuture.channel();
-  }
-
-  public void shutdown() {
-    shutdown(false);
-  }
-
-  public void shutdown(boolean waitUntilThreadsStop) {
-    try {
-      accepted.close();
-    } catch (Throwable t) {
-      LOG.error(t.getMessage(), t);
-    }
-
-    if(bootstrap != null) {
-      if (bootstrap.childGroup() != null) {
-        bootstrap.childGroup().shutdownGracefully();
-        if (waitUntilThreadsStop) {
-          bootstrap.childGroup().terminationFuture().awaitUninterruptibly();
-        }
-      }
-
-      if (bootstrap.group() != null) {
-        bootstrap.group().shutdownGracefully();
-        if (waitUntilThreadsStop) {
-          bootstrap.childGroup().terminationFuture().awaitUninterruptibly();
-        }
-      }
-    }
-
-    if (bindAddress != null) {
-      LOG.info("Rpc (" + serviceName + ") listened on "
-          + RpcUtils.normalizeInetSocketAddress(bindAddress)+ ") shutdown");
-    }
-  }
-
-  private static String getNextDefaultServiceName() {
-    return DEFAULT_PREFIX + sequenceId.getAndIncrement();
-  }
-
-  private static final int startPortRange = 10000;
-  private static final int endPortRange = 50000;
-  private static final Random rnd = new Random(System.currentTimeMillis());
-  // each system has a different starting port number within the given range.
-  private static final AtomicInteger nextPortNum =
-      new AtomicInteger(startPortRange+ rnd.nextInt(endPortRange - startPortRange));
-  private static final Object lockObject = new Object();
-
-
-  private synchronized static int getUnusedPort() throws IOException {
-    while (true) {
-      int port = nextPortNum.getAndIncrement();
-      if (port >= endPortRange) {
-        synchronized (lockObject) {
-          nextPortNum.set(startPortRange);
-          port = nextPortNum.getAndIncrement();
-        }
-      }
-      if (available(port)) {
-        return port;
-      }
-    }
-  }
-
-  private static boolean available(int port) throws IOException {
-    if (port < 1024 || port > 65535) {
-      throw new IllegalArgumentException("Port Number Out of Bound: " + port);
-    }
-
-    ServerSocket ss = null;
-    DatagramSocket ds = null;
-
-    try {
-      ss = new ServerSocket(port);
-      ss.setReuseAddress(true);
-
-      ds = new DatagramSocket(port);
-      ds.setReuseAddress(true);
-
-      return true;
-
-    } catch (IOException e) {
-      return false;
-    } finally {
-      if (ss != null) {
-        ss.close();
-      }
-
-      if (ds != null) {
-        ds.close();
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/NullCallback.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NullCallback.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NullCallback.java
deleted file mode 100644
index 9b7f8ac..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NullCallback.java
+++ /dev/null
@@ -1,38 +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.rpc;
-
-import com.google.protobuf.RpcCallback;
-
-public class NullCallback implements RpcCallback<Object> {
-  private final static NullCallback instance;
-
-  static {
-    instance = new NullCallback();
-  }
-
-  public static RpcCallback get() {
-    return instance;
-  }
-
-  @Override
-  public void run(Object parameter) {
-    // do nothing
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.java
deleted file mode 100644
index 6a340dc..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoChannelInitializer.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.rpc;
-
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelPipeline;
-import io.netty.handler.codec.protobuf.ProtobufDecoder;
-import io.netty.handler.codec.protobuf.ProtobufEncoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
-
-import com.google.protobuf.MessageLite;
-
-class ProtoChannelInitializer extends ChannelInitializer<Channel> {
-  private final MessageLite defaultInstance;
-  private final ChannelHandler handler;
-
-  public ProtoChannelInitializer(ChannelHandler handler, MessageLite defaultInstance) {
-    this.handler = handler;
-    this.defaultInstance = defaultInstance;
-  }
-
-  @Override
-  protected void initChannel(Channel channel) throws Exception {
-    ChannelPipeline pipeline = channel.pipeline();
-    pipeline.addLast("frameDecoder", new ProtobufVarint32FrameDecoder());
-    pipeline.addLast("protobufDecoder", new ProtobufDecoder(defaultInstance));
-    pipeline.addLast("frameEncoder", new ProtobufVarint32LengthFieldPrepender());
-    pipeline.addLast("protobufEncoder", new ProtobufEncoder());
-    pipeline.addLast("handler", handler);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
deleted file mode 100644
index 52ef31a..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
+++ /dev/null
@@ -1,69 +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.rpc;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import org.apache.tajo.rpc.RpcProtos.RpcResponse;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.io.Writer;
-
-public class RemoteCallException extends RemoteException {
-  private int seqId;
-  private String originExceptionClass;
-
-  public RemoteCallException(int seqId, MethodDescriptor methodDesc,
-                             Throwable t) {
-    super("Remote call error occurs when " + methodDesc.getFullName() + "is called:", t);
-    this.seqId = seqId;
-    if (t != null) {
-      originExceptionClass = t.getClass().getCanonicalName();
-    }
-  }
-
-  public RemoteCallException(int seqId, Throwable t) {
-    super(t);
-    this.seqId = seqId;
-    if (t != null) {
-      originExceptionClass = t.getClass().getCanonicalName();
-    }
-  }
-
-  public RpcResponse getResponse() {
-    RpcResponse.Builder builder = RpcResponse.newBuilder();
-    builder.setId(seqId);
-    if (getCause().getMessage() == null) {
-      builder.setErrorMessage(getCause().getClass().getName());
-    } else {
-      builder.setErrorMessage(getCause().getMessage());
-    }
-    builder.setErrorTrace(getStackTraceString(getCause()));
-    builder.setErrorClass(originExceptionClass);
-
-    return builder.build();
-  }
-
-  private static String getStackTraceString(Throwable aThrowable) {
-    final Writer result = new StringWriter();
-    final PrintWriter printWriter = new PrintWriter(result);
-    aThrowable.printStackTrace(printWriter);
-    return result.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteException.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteException.java
deleted file mode 100644
index 30c110d..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteException.java
+++ /dev/null
@@ -1,37 +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.rpc;
-
-public class RemoteException extends RuntimeException {
-  public RemoteException() {
-    super();
-  }
-
-  public RemoteException(String message) {
-    super(message);
-  }
-
-  public RemoteException(Throwable t) {
-    super(t);
-  }
-
-  public RemoteException(String message, Throwable t) {
-    super(message, t);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/a9ae3cab/tajo-rpc/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
deleted file mode 100644
index 3c054ad..0000000
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RetriesExhaustedException.java
+++ /dev/null
@@ -1,104 +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.rpc;
-
-import java.io.IOException;
-import java.util.Date;
-import java.util.List;
-
-public class RetriesExhaustedException extends RuntimeException {
-  private static final long serialVersionUID = 1876775844L;
-
-  public RetriesExhaustedException(final String msg) {
-    super(msg);
-  }
-
-  public RetriesExhaustedException(final String msg, final IOException e) {
-    super(msg, e);
-  }
-
-  /**
-   * Datastructure that allows adding more info around Throwable incident.
-   */
-  public static class ThrowableWithExtraContext {
-    private final Throwable t;
-    private final long when;
-    private final String extras;
-
-    public ThrowableWithExtraContext(final Throwable t, final long when,
-        final String extras) {
-      this.t = t;
-      this.when = when;
-      this.extras = extras;
-    }
- 
-    @Override
-    public String toString() {
-      return new Date(this.when).toString() + ", " + extras + ", " + t.toString();
-    }
-  }
-
-  /**
-   * Create a new RetriesExhaustedException from the list of prior failures.
-   * @param callableVitals Details from the {@link ServerCallable} we were using
-   * when we got this exception.
-   * @param numTries The number of tries we made
-   * @param exceptions List of exceptions that failed before giving up
-   */
-  public RetriesExhaustedException(final String callableVitals, int numTries,
-      List<Throwable> exceptions) {
-    super(getMessage(callableVitals, numTries, exceptions));
-  }
-
-  /**
-   * Create a new RetriesExhaustedException from the list of prior failures.
-   * @param numTries
-   * @param exceptions List of exceptions that failed before giving up
-   */
-  public RetriesExhaustedException(final int numTries,
-      final List<Throwable> exceptions) {
-    super(getMessage(numTries, exceptions));
-  }
-
-  private static String getMessage(String callableVitals, int numTries,
-      List<Throwable> exceptions) {
-    StringBuilder buffer = new StringBuilder("Failed contacting ");
-    buffer.append(callableVitals);
-    buffer.append(" after ");
-    buffer.append(numTries + 1);
-    buffer.append(" attempts.\nExceptions:\n");
-    for (Throwable t : exceptions) {
-      buffer.append(t.toString());
-      buffer.append("\n");
-    }
-    return buffer.toString();
-  }
-
-  private static String getMessage(final int numTries,
-      final List<Throwable> exceptions) {
-    StringBuilder buffer = new StringBuilder("Failed after attempts=");
-    buffer.append(numTries + 1);
-    buffer.append(", exceptions:\n");
-    for (Throwable t : exceptions) {
-      buffer.append(t.toString());
-      buffer.append("\n");
-    }
-    return buffer.toString();
-  }
-}
\ No newline at end of file