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