Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
3aeb00f6
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
未验证
提交
3aeb00f6
编写于
5月 07, 2018
作者:
S
Stephan Ewen
提交者:
Till Rohrmann
7月 16, 2018
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-9313] [security] (part 3) Activate mutual authentication for RPC/akka
上级
30c4bc84
变更
2
隐藏空白更改
内联
并排
Showing
2 changed file
with
163 addition
and
0 deletion
+163
-0
flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
.../main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
+1
-0
flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java
...t/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java
+162
-0
未找到文件。
flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
浏览文件 @
3aeb00f6
...
...
@@ -453,6 +453,7 @@ object AkkaUtils {
| protocol = $akkaSSLProtocol
| enabled-algorithms = $akkaSSLAlgorithms
| random-number-generator = ""
| require-mutual-authentication = on
| }
| }
| }
...
...
flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java
0 → 100644
浏览文件 @
3aeb00f6
/*
* 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.flink.runtime.rpc
;
import
org.apache.flink.api.common.time.Time
;
import
org.apache.flink.configuration.Configuration
;
import
org.apache.flink.configuration.SecurityOptions
;
import
org.apache.flink.runtime.akka.AkkaUtils
;
import
org.apache.flink.runtime.concurrent.FutureUtils
;
import
org.apache.flink.runtime.rpc.akka.AkkaRpcService
;
import
org.apache.flink.runtime.rpc.exceptions.RpcConnectionException
;
import
org.apache.flink.util.TestLogger
;
import
akka.actor.ActorSystem
;
import
akka.actor.Terminated
;
import
org.junit.Test
;
import
java.util.Arrays
;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.ExecutionException
;
import
java.util.concurrent.TimeUnit
;
import
static
org
.
junit
.
Assert
.
assertTrue
;
import
static
org
.
junit
.
Assert
.
fail
;
/**
* This test validates that the RPC service gives a good message when it cannot
* connect to an RpcEndpoint.
*/
public
class
RpcSSLAuthITCase
extends
TestLogger
{
private
static
final
String
KEY_STORE_FILE
=
RpcSSLAuthITCase
.
class
.
getResource
(
"/local127.keystore"
).
getFile
();
private
static
final
String
TRUST_STORE_FILE
=
RpcSSLAuthITCase
.
class
.
getResource
(
"/local127.truststore"
).
getFile
();
private
static
final
String
UNTRUSTED_KEY_STORE_FILE
=
RpcSSLAuthITCase
.
class
.
getResource
(
"/untrusted.keystore"
).
getFile
();
@Test
public
void
testConnectFailure
()
throws
Exception
{
// !!! This config has KEY_STORE_FILE / TRUST_STORE_FILE !!!
Configuration
sslConfig1
=
new
Configuration
();
sslConfig1
.
setBoolean
(
SecurityOptions
.
SSL_INTERNAL_ENABLED
,
true
);
sslConfig1
.
setString
(
SecurityOptions
.
SSL_INTERNAL_KEYSTORE
,
KEY_STORE_FILE
);
sslConfig1
.
setString
(
SecurityOptions
.
SSL_INTERNAL_TRUSTSTORE
,
TRUST_STORE_FILE
);
sslConfig1
.
setString
(
SecurityOptions
.
SSL_INTERNAL_KEYSTORE_PASSWORD
,
"password"
);
sslConfig1
.
setString
(
SecurityOptions
.
SSL_INTERNAL_KEY_PASSWORD
,
"password"
);
sslConfig1
.
setString
(
SecurityOptions
.
SSL_INTERNAL_TRUSTSTORE_PASSWORD
,
"password"
);
sslConfig1
.
setString
(
SecurityOptions
.
SSL_ALGORITHMS
,
"TLS_RSA_WITH_AES_128_CBC_SHA"
);
// !!! This config has KEY_STORE_FILE / UNTRUSTED_KEY_STORE_FILE !!!
// If this is presented by a client, it will trust the server, but the server will
// not trust this client in case client auth is enabled.
Configuration
sslConfig2
=
new
Configuration
();
sslConfig2
.
setBoolean
(
SecurityOptions
.
SSL_INTERNAL_ENABLED
,
true
);
sslConfig2
.
setString
(
SecurityOptions
.
SSL_INTERNAL_KEYSTORE
,
UNTRUSTED_KEY_STORE_FILE
);
sslConfig2
.
setString
(
SecurityOptions
.
SSL_INTERNAL_TRUSTSTORE
,
TRUST_STORE_FILE
);
sslConfig2
.
setString
(
SecurityOptions
.
SSL_INTERNAL_KEYSTORE_PASSWORD
,
"password"
);
sslConfig2
.
setString
(
SecurityOptions
.
SSL_INTERNAL_KEY_PASSWORD
,
"password"
);
sslConfig2
.
setString
(
SecurityOptions
.
SSL_INTERNAL_TRUSTSTORE_PASSWORD
,
"password"
);
sslConfig2
.
setString
(
SecurityOptions
.
SSL_ALGORITHMS
,
"TLS_RSA_WITH_AES_128_CBC_SHA"
);
ActorSystem
actorSystem1
=
null
;
ActorSystem
actorSystem2
=
null
;
RpcService
rpcService1
=
null
;
RpcService
rpcService2
=
null
;
try
{
actorSystem1
=
AkkaUtils
.
createActorSystem
(
sslConfig1
,
"localhost"
,
0
);
actorSystem2
=
AkkaUtils
.
createActorSystem
(
sslConfig2
,
"localhost"
,
0
);
// to test whether the test is still good:
// - create actorSystem2 with sslConfig1 (same as actorSystem1) and see that both can connect
// - set 'require-mutual-authentication = off' in the AkkaUtils ssl config section
// we start the RPC service with a very long timeout to ensure that the test
// can only pass if the connection problem is not recognized merely via a timeout
rpcService1
=
new
AkkaRpcService
(
actorSystem1
,
Time
.
of
(
10000000
,
TimeUnit
.
SECONDS
));
rpcService2
=
new
AkkaRpcService
(
actorSystem2
,
Time
.
of
(
10000000
,
TimeUnit
.
SECONDS
));
TestEndpoint
endpoint
=
new
TestEndpoint
(
rpcService1
);
endpoint
.
start
();
CompletableFuture
<
TestGateway
>
future
=
rpcService2
.
connect
(
endpoint
.
getAddress
(),
TestGateway
.
class
);
TestGateway
gateway
=
future
.
get
(
10000000
,
TimeUnit
.
SECONDS
);
CompletableFuture
<
String
>
fooFuture
=
gateway
.
foo
();
fooFuture
.
get
();
fail
(
"should never complete normally"
);
}
catch
(
ExecutionException
e
)
{
// that is what we want
assertTrue
(
e
.
getCause
()
instanceof
RpcConnectionException
);
}
finally
{
final
CompletableFuture
<
Void
>
rpcTerminationFuture1
=
rpcService1
!=
null
?
rpcService1
.
stopService
()
:
CompletableFuture
.
completedFuture
(
null
);
final
CompletableFuture
<
Void
>
rpcTerminationFuture2
=
rpcService2
!=
null
?
rpcService2
.
stopService
()
:
CompletableFuture
.
completedFuture
(
null
);
final
CompletableFuture
<
Terminated
>
actorSystemTerminationFuture1
=
actorSystem1
!=
null
?
FutureUtils
.
toJava
(
actorSystem1
.
terminate
())
:
CompletableFuture
.
completedFuture
(
null
);
final
CompletableFuture
<
Terminated
>
actorSystemTerminationFuture2
=
actorSystem2
!=
null
?
FutureUtils
.
toJava
(
actorSystem2
.
terminate
())
:
CompletableFuture
.
completedFuture
(
null
);
FutureUtils
.
waitForAll
(
Arrays
.
asList
(
rpcTerminationFuture1
,
rpcTerminationFuture2
,
actorSystemTerminationFuture1
,
actorSystemTerminationFuture2
))
.
get
();
}
}
// ------------------------------------------------------------------------
// Test RPC endpoint
// ------------------------------------------------------------------------
/** doc. */
public
interface
TestGateway
extends
RpcGateway
{
CompletableFuture
<
String
>
foo
();
}
/** doc. */
public
static
class
TestEndpoint
extends
RpcEndpoint
implements
TestGateway
{
public
TestEndpoint
(
RpcService
rpcService
)
{
super
(
rpcService
);
}
@Override
public
CompletableFuture
<
Void
>
postStop
()
{
return
CompletableFuture
.
completedFuture
(
null
);
}
@Override
public
CompletableFuture
<
String
>
foo
()
{
return
CompletableFuture
.
completedFuture
(
"bar"
);
}
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录