You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by ta...@apache.org on 2020/01/08 06:51:00 UTC

[skywalking] 01/01: add token authentication between agent and oap receiver.

This is an automated email from the ASF dual-hosted git repository.

tanjian pushed a commit to branch agent_auth
in repository https://gitbox.apache.org/repos/asf/skywalking.git

commit 47c54f88d01bb5a2656424d31e79b681c127d542
Author: JaredTan95 <ji...@daocloud.io>
AuthorDate: Wed Jan 8 14:42:06 2020 +0800

    add token authentication between agent and oap receiver.
---
 .../apm/agent/core/remote/GRPCChannel.java         |  4 +-
 .../apm/agent/core/remote/GRPCChannelManager.java  | 32 +++++-----
 docker/oap-es7/docker-entrypoint.sh                |  1 +
 docker/oap/docker-entrypoint.sh                    |  1 +
 docs/en/setup/backend/backend-token-auth.md        | 44 ++++++++++++++
 .../src/main/resources/application.yml             |  1 +
 .../core/server/GRPCHandlerRegisterImpl.java       |  6 +-
 .../core/server/auth/AuthenticationHandler.java    | 69 +++++++++++++++++++++
 .../library/server/grpc/GRPCServerConfig.java      | 71 ----------------------
 .../sharing/server/SharingServerConfig.java        |  1 +
 .../server/SharingServerModuleProvider.java        |  3 +-
 skywalking-ui                                      |  2 +-
 12 files changed, 140 insertions(+), 95 deletions(-)

diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannel.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannel.java
index 8e3afec..0f8e065 100755
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannel.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannel.java
@@ -91,8 +91,8 @@ public class GRPCChannel {
         private Builder(String host, int port) {
             this.host = host;
             this.port = port;
-            this.channelBuilders = new LinkedList<ChannelBuilder>();
-            this.decorators = new LinkedList<ChannelDecorator>();
+            this.channelBuilders = new LinkedList<>();
+            this.decorators = new LinkedList<>();
         }
 
         public Builder addChannelDecorator(ChannelDecorator interceptor) {
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannelManager.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannelManager.java
index 6aed148..97e482f 100755
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannelManager.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/GRPCChannelManager.java
@@ -68,21 +68,17 @@ public class GRPCChannelManager implements BootService, Runnable {
         grpcServers = Arrays.asList(Config.Collector.BACKEND_SERVICE.split(","));
         connectCheckFuture = Executors
             .newSingleThreadScheduledExecutor(new DefaultNamedThreadFactory("GRPCChannelManager"))
-            .scheduleAtFixedRate(new RunnableWithExceptionProtection(this, new RunnableWithExceptionProtection.CallbackWhenException() {
-                @Override
-                public void handle(Throwable t) {
-                    logger.error("unexpected exception.", t);
-                }
-            }), 0, Config.Collector.GRPC_CHANNEL_CHECK_INTERVAL, TimeUnit.SECONDS);
+            .scheduleAtFixedRate(new RunnableWithExceptionProtection(this,
+                t -> logger.error("unexpected exception.", t)), 0, Config.Collector.GRPC_CHANNEL_CHECK_INTERVAL, TimeUnit.SECONDS);
     }
 
     @Override
-    public void onComplete() throws Throwable {
+    public void onComplete() {
 
     }
 
     @Override
-    public void shutdown() throws Throwable {
+    public void shutdown() {
         if (connectCheckFuture != null) {
             connectCheckFuture.cancel(true);
         }
@@ -111,11 +107,11 @@ public class GRPCChannelManager implements BootService, Runnable {
                         }
 
                         managedChannel = GRPCChannel.newBuilder(ipAndPort[0], Integer.parseInt(ipAndPort[1]))
-                            .addManagedChannelBuilder(new StandardChannelBuilder())
-                            .addManagedChannelBuilder(new TLSChannelBuilder())
-                            .addChannelDecorator(new AgentIDDecorator())
-                            .addChannelDecorator(new AuthenticationDecorator())
-                            .build();
+                                .addManagedChannelBuilder(new StandardChannelBuilder())
+                                .addManagedChannelBuilder(new TLSChannelBuilder())
+                                .addChannelDecorator(new AgentIDDecorator())
+                                .addChannelDecorator(new AuthenticationDecorator())
+                                .build();
                         notify(GRPCChannelStatus.CONNECTED);
                         reconnectCount = 0;
                         reconnect = false;
@@ -172,11 +168,11 @@ public class GRPCChannelManager implements BootService, Runnable {
         if (throwable instanceof StatusRuntimeException) {
             StatusRuntimeException statusRuntimeException = (StatusRuntimeException) throwable;
             return statusEquals(statusRuntimeException.getStatus(),
-                Status.UNAVAILABLE,
-                Status.PERMISSION_DENIED,
-                Status.UNAUTHENTICATED,
-                Status.RESOURCE_EXHAUSTED,
-                Status.UNKNOWN
+                    Status.UNAVAILABLE,
+                    Status.PERMISSION_DENIED,
+                    Status.UNAUTHENTICATED,
+                    Status.RESOURCE_EXHAUSTED,
+                    Status.UNKNOWN
             );
         }
         return false;
diff --git a/docker/oap-es7/docker-entrypoint.sh b/docker/oap-es7/docker-entrypoint.sh
index aca6994..b1ab944 100755
--- a/docker/oap-es7/docker-entrypoint.sh
+++ b/docker/oap-es7/docker-entrypoint.sh
@@ -338,6 +338,7 @@ receiver-sharing-server:
    maxMessageSize: \${SW_RECEIVER_SHARING_MAX_MESSAGE_SIZE:0}
    gRPCThreadPoolSize: \${SW_RECEIVER_SHARING_GRPC_THREAD_POOL_SIZE:0}
    gRPCThreadPoolQueueSize: \${SW_RECEIVER_SHARING_GRPC_THREAD_POOL_QUEUE_SIZE:0}
+   authentication: \${SW_AUTHENTICATION:""}
 receiver-register:
   default:
 receiver-trace:
diff --git a/docker/oap/docker-entrypoint.sh b/docker/oap/docker-entrypoint.sh
index 8ffee25..7761f70 100755
--- a/docker/oap/docker-entrypoint.sh
+++ b/docker/oap/docker-entrypoint.sh
@@ -339,6 +339,7 @@ receiver-sharing-server:
    maxMessageSize: \${SW_RECEIVER_SHARING_MAX_MESSAGE_SIZE:0}
    gRPCThreadPoolSize: \${SW_RECEIVER_SHARING_GRPC_THREAD_POOL_SIZE:0}
    gRPCThreadPoolQueueSize: \${SW_RECEIVER_SHARING_GRPC_THREAD_POOL_QUEUE_SIZE:0}
+   authentication: \${SW_AUTHENTICATION:""}
 receiver-register:
   default:
 receiver-trace:
diff --git a/docs/en/setup/backend/backend-token-auth.md b/docs/en/setup/backend/backend-token-auth.md
index e69de29..b1fbc85 100644
--- a/docs/en/setup/backend/backend-token-auth.md
+++ b/docs/en/setup/backend/backend-token-auth.md
@@ -0,0 +1,44 @@
+# Token Authentication
+## Supported version
+7.0.0+
+
+## How need token authentication after we have TLS?
+TLS is about transport security, which makes sure the network can be trusted. 
+The token authentication is about monitoring application data **can be trusted**.
+
+## Token 
+In current version, Token is considered as a simple string.
+
+### Set Token
+1. Set token in agent.config file
+```properties
+# Authentication active is based on backend setting, see application.yml for more details.
+agent.authentication = ${SW_AGENT_AUTHENTICATION:xxxx}
+```
+
+2. Set token in `application.yml` file
+```yaml
+······
+receiver-sharing-server:
+  default:
+    authentication: ${SW_AUTHENTICATION:""}
+······
+```
+
+## Authentication fails
+The Skywalking OAP verifies every request from agent, allowed only the token match.
+
+If the token is not right, you will see the following log in agent
+```
+org.apache.skywalking.apm.dependencies.io.grpc.StatusRuntimeException: PERMISSION_DENIED
+```
+
+## FAQ
+### Can I use token authentication instead of TLS?
+No, you shouldn't. In tech way, you can of course, but token and TLS are used for untrusted network env. In that circumstance,
+TLS has higher priority than this. Token can be trusted only under TLS protection.Token can be stolen easily if you 
+send it through a non-TLS network.
+
+### Do you support other authentication mechanisms? Such as ak/sk?
+For now, no. But we appreciate someone contributes this feature. 
+
diff --git a/oap-server/server-bootstrap/src/main/resources/application.yml b/oap-server/server-bootstrap/src/main/resources/application.yml
index c2281ff..3f79711 100755
--- a/oap-server/server-bootstrap/src/main/resources/application.yml
+++ b/oap-server/server-bootstrap/src/main/resources/application.yml
@@ -137,6 +137,7 @@ storage:
 #    metadataQueryMaxSize: ${SW_STORAGE_MYSQL_QUERY_MAX_SIZE:5000}
 receiver-sharing-server:
   default:
+    authentication: ${SW_AUTHENTICATION:""}
 receiver-register:
   default:
 receiver-trace:
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/server/GRPCHandlerRegisterImpl.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/server/GRPCHandlerRegisterImpl.java
index 229c3ca..5985708 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/server/GRPCHandlerRegisterImpl.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/server/GRPCHandlerRegisterImpl.java
@@ -18,7 +18,9 @@
 
 package org.apache.skywalking.oap.server.core.server;
 
-import io.grpc.*;
+import io.grpc.BindableService;
+import io.grpc.ServerServiceDefinition;
+import org.apache.skywalking.oap.server.core.server.auth.AuthenticationHandler;
 import org.apache.skywalking.oap.server.library.server.grpc.GRPCServer;
 
 /**
@@ -33,7 +35,7 @@ public class GRPCHandlerRegisterImpl implements GRPCHandlerRegister {
     }
 
     @Override public void addHandler(BindableService handler) {
-        server.addHandler(handler);
+        AuthenticationHandler.INSTANCE.build(server, handler);
     }
 
     @Override public void addHandler(ServerServiceDefinition definition) {
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/server/auth/AuthenticationHandler.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/server/auth/AuthenticationHandler.java
new file mode 100644
index 0000000..404261d
--- /dev/null
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/server/auth/AuthenticationHandler.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.skywalking.oap.server.core.server.auth;
+
+import io.grpc.BindableService;
+import io.grpc.Metadata;
+import io.grpc.ServerCall;
+import io.grpc.ServerCallHandler;
+import io.grpc.ServerInterceptor;
+import io.grpc.ServerInterceptors;
+import io.grpc.Status;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.server.GRPCHandlerRegister;
+import org.apache.skywalking.oap.server.library.server.grpc.GRPCServer;
+
+/**
+ * Active the authentication(agent <---> oap) token checker if expected token exists in application.yml
+ *
+ * @author wusheng, jian.tan
+ */
+public enum AuthenticationHandler {
+    INSTANCE;
+
+    private static final Metadata.Key<String> AUTH_HEAD_HEADER_NAME = Metadata.Key.of("Authentication", Metadata.ASCII_STRING_MARSHALLER);
+
+    private String expectedToken = "";
+
+    public void build(GRPCServer grpcHandler, BindableService targetService) {
+        if (StringUtil.isNotEmpty(expectedToken)) {
+            grpcHandler.addHandler(ServerInterceptors.intercept(targetService, new ServerInterceptor() {
+                @Override
+                public <REQ, RESP> ServerCall.Listener<REQ> interceptCall(ServerCall<REQ, RESP> serverCall,
+                    Metadata metadata,
+                    ServerCallHandler<REQ, RESP> next) {
+                    String token = metadata.get(AUTH_HEAD_HEADER_NAME);
+                    if (expectedToken.equals(token)) {
+                        return next.startCall(serverCall, metadata);
+                    } else {
+                        serverCall.close(Status.PERMISSION_DENIED, new Metadata());
+                        return new ServerCall.Listener() {
+                        };
+                    }
+                }
+            }));
+        } else {
+            grpcHandler.addHandler(targetService);
+        }
+    }
+
+    public void setExpectedToken(String expectedToken) {
+        this.expectedToken = expectedToken;
+    }
+}
\ No newline at end of file
diff --git a/oap-server/server-library/library-server/src/main/java/org/apache/skywalking/oap/server/library/server/grpc/GRPCServerConfig.java b/oap-server/server-library/library-server/src/main/java/org/apache/skywalking/oap/server/library/server/grpc/GRPCServerConfig.java
deleted file mode 100644
index f0a5c2f..0000000
--- a/oap-server/server-library/library-server/src/main/java/org/apache/skywalking/oap/server/library/server/grpc/GRPCServerConfig.java
+++ /dev/null
@@ -1,71 +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.skywalking.oap.server.library.server.grpc;
-
-/**
- * @author peng-yongsheng
- */
-public abstract class GRPCServerConfig {
-
-    private String host;
-    private int port;
-    private String sslCertChainFilePath;
-    private String sslPrivateKeyFilePath;
-    private String authentication;
-
-    public String getHost() {
-        return host;
-    }
-
-    public void setHost(String host) {
-        this.host = host;
-    }
-
-    public int getPort() {
-        return port;
-    }
-
-    public void setPort(int port) {
-        this.port = port;
-    }
-
-    public String getSslCertChainFilePath() {
-        return sslCertChainFilePath;
-    }
-
-    public void setSslCertChainFilePath(String sslCertChainFilePath) {
-        this.sslCertChainFilePath = sslCertChainFilePath;
-    }
-
-    public String getSslPrivateKeyFilePath() {
-        return sslPrivateKeyFilePath;
-    }
-
-    public void setSslPrivateKeyFilePath(String sslPrivateKeyFilePath) {
-        this.sslPrivateKeyFilePath = sslPrivateKeyFilePath;
-    }
-
-    public String getAuthentication() {
-        return authentication;
-    }
-
-    public void setAuthentication(String authentication) {
-        this.authentication = authentication;
-    }
-}
diff --git a/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerConfig.java b/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerConfig.java
index 2f0b005..bfd2155 100644
--- a/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerConfig.java
+++ b/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerConfig.java
@@ -36,4 +36,5 @@ public class SharingServerConfig extends ModuleConfig {
     private int maxMessageSize;
     private int gRPCThreadPoolSize;
     private int gRPCThreadPoolQueueSize;
+    private String authentication;
 }
diff --git a/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerModuleProvider.java b/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerModuleProvider.java
index a4ae401..87f879d 100644
--- a/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerModuleProvider.java
+++ b/oap-server/server-receiver-plugin/skywalking-sharing-server-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/sharing/server/SharingServerModuleProvider.java
@@ -23,6 +23,7 @@ import org.apache.logging.log4j.util.Strings;
 import org.apache.skywalking.oap.server.core.CoreModule;
 import org.apache.skywalking.oap.server.core.remote.health.HealthCheckServiceHandler;
 import org.apache.skywalking.oap.server.core.server.*;
+import org.apache.skywalking.oap.server.core.server.auth.AuthenticationHandler;
 import org.apache.skywalking.oap.server.library.module.*;
 import org.apache.skywalking.oap.server.library.server.ServerException;
 import org.apache.skywalking.oap.server.library.server.grpc.GRPCServer;
@@ -57,6 +58,7 @@ public class SharingServerModuleProvider extends ModuleProvider {
     }
 
     @Override public void prepare() {
+        AuthenticationHandler.INSTANCE.setExpectedToken(config.getAuthentication());
         if (config.getRestPort() != 0) {
             jettyServer = new JettyServer(Strings.isBlank(config.getRestHost()) ? "0.0.0.0" : config.getRestHost(), config.getRestPort(), config.getRestContextPath());
             jettyServer.initialize();
@@ -82,7 +84,6 @@ public class SharingServerModuleProvider extends ModuleProvider {
                 grpcServer.setThreadPoolSize(config.getGRPCThreadPoolSize());
             }
             grpcServer.initialize();
-
             this.registerServiceImplementation(GRPCHandlerRegister.class, new GRPCHandlerRegisterImpl(grpcServer));
         } else {
             this.receiverGRPCHandlerRegister = new ReceiverGRPCHandlerRegister();
diff --git a/skywalking-ui b/skywalking-ui
index 8cf4b86..f5d65ee 160000
--- a/skywalking-ui
+++ b/skywalking-ui
@@ -1 +1 @@
-Subproject commit 8cf4b8645f1adb5c3e5f99110671abe7a0fdbb7e
+Subproject commit f5d65eef9f027e306d0b6dc8a26e159a84c57234