diff --git a/lib/vnet/admin_process_windows.go b/lib/vnet/admin_process_windows.go
index 1c30c38eb36d1..73128560e9f86 100644
--- a/lib/vnet/admin_process_windows.go
+++ b/lib/vnet/admin_process_windows.go
@@ -18,16 +18,32 @@ package vnet
import (
"context"
+ "errors"
+ "os"
+ "syscall"
+ "time"
+ "unsafe"
+ "github.com/google/uuid"
"github.com/gravitational/trace"
+ "github.com/jonboulle/clockwork"
+ "golang.org/x/sync/errgroup"
+ "golang.org/x/sys/windows"
"golang.zx2c4.com/wireguard/tun"
)
+type windowsAdminProcessConfig struct {
+ clientApplicationServiceAddr string
+}
+
// runWindowsAdminProcess must run as administrator. It creates and sets up a TUN
// device, runs the VNet networking stack, and handles OS configuration. It will
// continue to run until [ctx] is canceled or encountering an unrecoverable
// error.
-func runWindowsAdminProcess(ctx context.Context) error {
+func runWindowsAdminProcess(ctx context.Context, cfg *windowsAdminProcessConfig) error {
+ pm, ctx := newProcessManager()
+ log.InfoContext(ctx, "Running VNet admin process")
+
device, err := tun.CreateTUN("TeleportVNet", mtu)
if err != nil {
return trace.Wrap(err, "creating TUN device")
@@ -38,8 +54,218 @@ func runWindowsAdminProcess(ctx context.Context) error {
return trace.Wrap(err, "getting TUN device name")
}
log.InfoContext(ctx, "Created TUN interface", "tun", tunName)
- // TODO(nklaassen): actually run VNet. For now, just stay alive until the
- // context is canceled.
- <-ctx.Done()
- return trace.Wrap(ctx.Err())
+
+ clt, err := newClientApplicationServiceClient(ctx, cfg.clientApplicationServiceAddr)
+ if err != nil {
+ return trace.Wrap(err, "creating user process client")
+ }
+ defer clt.Close()
+
+ if err := authenticateUserProcess(ctx, clt); err != nil {
+ log.ErrorContext(ctx, "Failed to authenticate user process", "error", err)
+ return trace.Wrap(err, "authenticating user process")
+ }
+
+ networkStackConfig, err := newWindowsNetworkStackConfig(device, clt)
+ if err != nil {
+ return trace.Wrap(err, "creating network stack config")
+ }
+ networkStack, err := newNetworkStack(networkStackConfig)
+ if err != nil {
+ return trace.Wrap(err, "creating network stack")
+ }
+
+ pm.AddCriticalBackgroundTask("network stack", func() error {
+ return trace.Wrap(networkStack.run(ctx), "running network stack")
+ })
+ pm.AddCriticalBackgroundTask("user process ping", func() error {
+ for {
+ select {
+ case <-time.After(time.Second):
+ if err := clt.Ping(ctx); err != nil {
+ return trace.Wrap(err, "pinging user process")
+ }
+ case <-ctx.Done():
+ return ctx.Err()
+ }
+ }
+ })
+ // TODO(nklaassen): run OS configuration loop.
+ return trace.Wrap(pm.Wait())
+}
+
+func newWindowsNetworkStackConfig(tun tunDevice, clt *clientApplicationServiceClient) (*networkStackConfig, error) {
+ appProvider := newRemoteAppProvider(clt)
+ appResolver := newTCPAppResolver(appProvider, clockwork.NewRealClock())
+ ipv6Prefix, err := NewIPv6Prefix()
+ if err != nil {
+ return nil, trace.Wrap(err, "creating new IPv6 prefix")
+ }
+ dnsIPv6 := ipv6WithSuffix(ipv6Prefix, []byte{2})
+ return &networkStackConfig{
+ tunDevice: tun,
+ ipv6Prefix: ipv6Prefix,
+ dnsIPv6: dnsIPv6,
+ tcpHandlerResolver: appResolver,
+ }, nil
+}
+
+func authenticateUserProcess(ctx context.Context, clt *clientApplicationServiceClient) error {
+ pipe, err := createNamedPipe(ctx)
+ if err != nil {
+ return trace.Wrap(err, "creating named pipe")
+ }
+ defer pipe.Close()
+ g, ctx := errgroup.WithContext(ctx)
+ g.Go(func() error {
+ if err := clt.AuthenticateProcess(ctx, pipe.name); err != nil {
+ return trace.Wrap(err, "authenticating user process")
+ }
+ return nil
+ })
+ g.Go(func() error {
+ if err := pipe.validateClientExe(ctx); err != nil {
+ return trace.Wrap(err, "validating user process exe")
+ }
+ return nil
+ })
+ if err := g.Wait(); err != nil {
+ return trace.Wrap(err)
+ }
+ return nil
+}
+
+type winpipe struct {
+ pipeHandle windows.Handle
+ eventHandle windows.Handle
+ name string
+}
+
+func createNamedPipe(ctx context.Context) (*winpipe, error) {
+ pipeName := `\\.\pipe\` + uuid.NewString()
+ pipePath, err := syscall.UTF16PtrFromString(pipeName)
+ if err != nil {
+ return nil, trace.Wrap(err, "converting string to UTF16")
+ }
+ // This allows pipe access to everyone
+ // TODO(nklaassen): restrict access to only the calling user.
+ sddl := "D:P(A;;GA;;;WD)"
+ sd, err := windows.SecurityDescriptorFromString(sddl)
+ if err != nil {
+ return nil, trace.Wrap(err, "creating security descriptor from string")
+ }
+ sa := windows.SecurityAttributes{
+ Length: uint32(unsafe.Sizeof(windows.SecurityAttributes{})),
+ SecurityDescriptor: sd,
+ InheritHandle: 0,
+ }
+ pipeHandle, err := windows.CreateNamedPipe(
+ pipePath,
+ windows.PIPE_ACCESS_DUPLEX|windows.FILE_FLAG_OVERLAPPED,
+ windows.PIPE_TYPE_BYTE|windows.PIPE_WAIT,
+ windows.PIPE_UNLIMITED_INSTANCES,
+ 1024,
+ 1024,
+ 0,
+ &sa,
+ )
+ if err != nil {
+ return nil, trace.Wrap(err, "creating named pipe")
+ }
+ log.DebugContext(ctx, "Created named pipe", "name", pipeName)
+ eventHandle, err := windows.CreateEvent(nil, 1, 0, nil)
+ if err != nil {
+ return nil, trace.Wrap(err, "creating Windows event handle")
+ }
+ overlapped := &windows.Overlapped{HEvent: eventHandle}
+ if err := windows.ConnectNamedPipe(pipeHandle, overlapped); err != nil && !errors.Is(err, windows.ERROR_IO_PENDING) {
+ return nil, trace.Wrap(err, "connecting to named pipe")
+ }
+ return &winpipe{
+ pipeHandle: pipeHandle,
+ eventHandle: overlapped.HEvent,
+ name: pipeName,
+ }, nil
+}
+
+func (p *winpipe) validateClientExe(ctx context.Context) error {
+ if err := p.waitForClient(ctx); err != nil {
+ return trace.Wrap(err, "waiting for client to connect to named pipe")
+ }
+ clientExePath, err := p.clientExePath(ctx)
+ if err != nil {
+ return trace.Wrap(err, "getting pipe client exe path")
+ }
+ log.DebugContext(ctx, "Got pipe connection from client", "exe", clientExePath)
+ thisExePath, err := os.Executable()
+ if err != nil {
+ return trace.Wrap(err, "getting executable path for this service")
+ }
+ if thisExePath != clientExePath {
+ return trace.AccessDenied("remote process is not running the same executable as this service, remote process exe: %s, this process exe: %s",
+ clientExePath, thisExePath)
+ }
+ // TODO(nklaassen): validate exe is signed, or consider if this is
+ // unnecessary as long as the two exes are identical.
+ return nil
+}
+
+func (p *winpipe) waitForClient(ctx context.Context) error {
+ evt, err := windows.WaitForSingleObject(p.eventHandle, 500 /*milliseconds*/)
+ if err != nil {
+ return trace.Wrap(err, "waiting for connection on named pipe")
+ }
+ if evt != windows.WAIT_OBJECT_0 {
+ return trace.Errorf("failed to wait for connection on named pipe, error code: %d", evt)
+ }
+ return nil
+}
+
+func (p *winpipe) clientExePath(ctx context.Context) (string, error) {
+ var pid uint32
+ if err := windows.GetNamedPipeClientProcessId(p.pipeHandle, &pid); err != nil {
+ return "", trace.Wrap(err, "getting named pipe client process ID")
+ }
+ processHandle, err := windows.OpenProcess(windows.PROCESS_QUERY_LIMITED_INFORMATION, false, pid)
+ if err != nil {
+ return "", trace.Wrap(err, "opening client process")
+ }
+ buf := make([]uint16, windows.MAX_PATH)
+ size := uint32(len(buf))
+ if err := windows.QueryFullProcessImageName(processHandle, 0, &buf[0], &size); err != nil {
+ return "", trace.Wrap(err, "querying pipe client process image name")
+ }
+ return windows.UTF16PtrToString(&buf[0]), nil
+}
+
+func (p *winpipe) Close() error {
+ return trace.NewAggregate(
+ trace.Wrap(windows.CloseHandle(p.pipeHandle), "closing pipe handle"),
+ trace.Wrap(windows.CloseHandle(p.eventHandle), "closing pipe event handle"),
+ )
+}
+
+// connectToPipe connects to a Windows named pipe, then immediately closes the
+// connection. This is used for process authentication.
+func connectToPipe(pipePath string) error {
+ pipePathPtr, err := syscall.UTF16PtrFromString(pipePath)
+ if err != nil {
+ return trace.Wrap(err, "converting string to UTF16")
+ }
+ handle, err := windows.CreateFile(
+ pipePathPtr,
+ windows.GENERIC_READ|windows.GENERIC_WRITE,
+ 0, // ShareMode
+ nil, // SecurityAttributes
+ windows.OPEN_EXISTING,
+ 0, // FlagsAndAttributes
+ 0, // TemplateFile
+ )
+ if err != nil {
+ return trace.Wrap(err, "opening named pipe")
+ }
+ if err := windows.CloseHandle(handle); err != nil {
+ return trace.Wrap(err, "closing named pipe")
+ }
+ return nil
}
diff --git a/lib/vnet/client_application_service.go b/lib/vnet/client_application_service.go
new file mode 100644
index 0000000000000..e8c1e555b90c6
--- /dev/null
+++ b/lib/vnet/client_application_service.go
@@ -0,0 +1,169 @@
+// Teleport
+// Copyright (C) 2024 Gravitational, Inc.
+//
+// This program is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Affero General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// This program is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Affero General Public License for more details.
+//
+// You should have received a copy of the GNU Affero General Public License
+// along with this program. If not, see .
+
+package vnet
+
+import (
+ "context"
+ "crypto"
+ "crypto/rand"
+ "sync"
+
+ "github.com/gravitational/trace"
+
+ "github.com/gravitational/teleport/api"
+ "github.com/gravitational/teleport/api/trail"
+ vnetv1 "github.com/gravitational/teleport/gen/proto/go/teleport/lib/vnet/v1"
+)
+
+type clientApplicationService struct {
+ // opt-in to compilation errors if this doesn't implement
+ // [vnetv1.ClientApplicationServiceServer]
+ vnetv1.UnsafeClientApplicationServiceServer
+
+ appProvider appProvider
+
+ mu sync.Mutex
+ // appSignerCache caches the crypto.Signer for each certificate issued by
+ // ReissueAppCert so that SignForApp can later use that signer.
+ appSignerCache map[appKey]crypto.Signer
+}
+
+func newClientApplicationService(appProvider appProvider) *clientApplicationService {
+ return &clientApplicationService{
+ appProvider: appProvider,
+ appSignerCache: make(map[appKey]crypto.Signer),
+ }
+}
+
+func (s *clientApplicationService) Ping(ctx context.Context, req *vnetv1.PingRequest) (*vnetv1.PingResponse, error) {
+ return &vnetv1.PingResponse{}, nil
+}
+
+func (s *clientApplicationService) AuthenticateProcess(ctx context.Context, req *vnetv1.AuthenticateProcessRequest) (*vnetv1.AuthenticateProcessResponse, error) {
+ log.DebugContext(ctx, "Received AuthenticateProcess request from admin process")
+ if req.Version != api.Version {
+ return nil, trace.BadParameter("version mismatch, user process version is %s, admin process version is %s",
+ api.Version, req.Version)
+ }
+ if err := platformAuthenticateProcess(ctx, req); err != nil {
+ return nil, trail.ToGRPC(err)
+ }
+ return &vnetv1.AuthenticateProcessResponse{
+ Version: api.Version,
+ }, nil
+}
+
+func (s *clientApplicationService) ResolveAppInfo(ctx context.Context, req *vnetv1.ResolveAppInfoRequest) (*vnetv1.ResolveAppInfoResponse, error) {
+ appInfo, err := s.appProvider.ResolveAppInfo(ctx, req.GetFqdn())
+ if err != nil {
+ return nil, trail.ToGRPC(err)
+ }
+ return &vnetv1.ResolveAppInfoResponse{
+ AppInfo: appInfo,
+ }, nil
+}
+
+func (s *clientApplicationService) ReissueAppCert(ctx context.Context, req *vnetv1.ReissueAppCertRequest) (*vnetv1.ReissueAppCertResponse, error) {
+ if req.AppInfo == nil {
+ return nil, trail.ToGRPC(trace.BadParameter("missing AppInfo"))
+ }
+ cert, err := s.appProvider.ReissueAppCert(ctx, req.GetAppInfo(), uint16(req.GetTargetPort()))
+ if err != nil {
+ return nil, trail.ToGRPC(trace.Wrap(err, "reissuing app certificate"))
+ }
+ s.setSignerForApp(req.GetAppInfo().GetAppKey(), uint16(req.GetTargetPort()), cert.PrivateKey.(crypto.Signer))
+ return &vnetv1.ReissueAppCertResponse{
+ Cert: cert.Certificate[0],
+ }, nil
+}
+
+func (s *clientApplicationService) SignForApp(ctx context.Context, req *vnetv1.SignForAppRequest) (*vnetv1.SignForAppResponse, error) {
+ log.DebugContext(ctx, "Got SignForApp request",
+ "app", req.GetAppKey(),
+ "hash", req.GetHash(),
+ "digest_len", len(req.GetDigest()),
+ )
+ var hash crypto.Hash
+ switch req.GetHash() {
+ case vnetv1.Hash_HASH_NONE:
+ hash = crypto.Hash(0)
+ case vnetv1.Hash_HASH_SHA256:
+ hash = crypto.SHA256
+ default:
+ return nil, trail.ToGRPC(trace.BadParameter("unsupported hash %v", req.GetHash()))
+ }
+ appKey := req.GetAppKey()
+
+ signer, ok := s.getSignerForApp(req.GetAppKey(), uint16(req.GetTargetPort()))
+ if !ok {
+ return nil, trail.ToGRPC(trace.BadParameter("no signer for app %v", appKey))
+ }
+
+ signature, err := signer.Sign(rand.Reader, req.GetDigest(), hash)
+ if err != nil {
+ return nil, trail.ToGRPC(trace.Wrap(err, "signing for app %v", appKey))
+ }
+ return &vnetv1.SignForAppResponse{
+ Signature: signature,
+ }, nil
+}
+
+func (s *clientApplicationService) setSignerForApp(appKey *vnetv1.AppKey, targetPort uint16, signer crypto.Signer) {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ s.appSignerCache[newAppKey(appKey, targetPort)] = signer
+}
+
+func (s *clientApplicationService) getSignerForApp(appKey *vnetv1.AppKey, targetPort uint16) (crypto.Signer, bool) {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ signer, ok := s.appSignerCache[newAppKey(appKey, targetPort)]
+ return signer, ok
+}
+
+// OnNewConnection gets called whenever a new connection is about to be
+// established through VNet for observability.
+func (s *clientApplicationService) OnNewConnection(ctx context.Context, req *vnetv1.OnNewConnectionRequest) (*vnetv1.OnNewConnectionResponse, error) {
+ if err := s.appProvider.OnNewConnection(ctx, req.GetAppKey()); err != nil {
+ return nil, trail.ToGRPC(err)
+ }
+ return &vnetv1.OnNewConnectionResponse{}, nil
+}
+
+// OnInvalidLocalPort gets called before VNet refuses to handle a connection
+// to a multi-port TCP app because the provided port does not match any of the
+// TCP ports in the app spec.
+func (s *clientApplicationService) OnInvalidLocalPort(ctx context.Context, req *vnetv1.OnInvalidLocalPortRequest) (*vnetv1.OnInvalidLocalPortResponse, error) {
+ s.appProvider.OnInvalidLocalPort(ctx, req.GetAppInfo(), uint16(req.GetTargetPort()))
+ return &vnetv1.OnInvalidLocalPortResponse{}, nil
+}
+
+// appKey is a clone of [vnetv1.AppKey] that is not a protobuf type so it can be
+// used as a key in maps.
+type appKey struct {
+ profile, leafCluster, app string
+ port uint16
+}
+
+func newAppKey(protoAppKey *vnetv1.AppKey, port uint16) appKey {
+ return appKey{
+ profile: protoAppKey.GetProfile(),
+ leafCluster: protoAppKey.GetLeafCluster(),
+ app: protoAppKey.GetName(),
+ port: port,
+ }
+}
diff --git a/lib/vnet/client_application_service_client.go b/lib/vnet/client_application_service_client.go
new file mode 100644
index 0000000000000..0a52193a2a0b8
--- /dev/null
+++ b/lib/vnet/client_application_service_client.go
@@ -0,0 +1,112 @@
+package vnet
+
+import (
+ "context"
+
+ "github.com/gravitational/teleport/api"
+ "github.com/gravitational/teleport/api/trail"
+ "github.com/gravitational/teleport/api/utils/grpc/interceptors"
+ vnetv1 "github.com/gravitational/teleport/gen/proto/go/teleport/lib/vnet/v1"
+ "github.com/gravitational/trace"
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/credentials/insecure"
+)
+
+// clientApplicationServiceClient is a client for the client application
+// service. This client is used in the Windows admin service to make requests to
+// the VNet client application.
+type clientApplicationServiceClient struct {
+ clt vnetv1.ClientApplicationServiceClient
+ conn *grpc.ClientConn
+}
+
+func newClientApplicationServiceClient(ctx context.Context, addr string) (*clientApplicationServiceClient, error) {
+ conn, err := grpc.NewClient(addr,
+ grpc.WithTransportCredentials(insecure.NewCredentials()),
+ grpc.WithUnaryInterceptor(interceptors.GRPCClientUnaryErrorInterceptor),
+ grpc.WithStreamInterceptor(interceptors.GRPCClientStreamErrorInterceptor),
+ )
+ if err != nil {
+ return nil, trace.Wrap(err, "creating user process gRPC client")
+ }
+ return &clientApplicationServiceClient{
+ clt: vnetv1.NewClientApplicationServiceClient(conn),
+ conn: conn,
+ }, nil
+}
+
+func (c *clientApplicationServiceClient) Close() error {
+ return trace.Wrap(c.conn.Close())
+}
+
+func (c *clientApplicationServiceClient) Ping(ctx context.Context) error {
+ if _, err := c.clt.Ping(ctx, &vnetv1.PingRequest{}); err != nil {
+ return trail.FromGRPC(err)
+ }
+ return nil
+}
+
+func (c *clientApplicationServiceClient) AuthenticateProcess(ctx context.Context, pipePath string) error {
+ resp, err := c.clt.AuthenticateProcess(ctx, &vnetv1.AuthenticateProcessRequest{
+ Version: api.Version,
+ PipePath: pipePath,
+ })
+ if err != nil {
+ return trail.FromGRPC(err)
+ }
+ if resp.Version != api.Version {
+ return trace.BadParameter("version mismatch, user process version is %s, admin process version is %s",
+ resp.Version, api.Version)
+ }
+ return nil
+}
+
+func (c *clientApplicationServiceClient) ResolveAppInfo(ctx context.Context, fqdn string) (*vnetv1.AppInfo, error) {
+ resp, err := c.clt.ResolveAppInfo(ctx, &vnetv1.ResolveAppInfoRequest{
+ Fqdn: fqdn,
+ })
+ if err != nil {
+ return nil, trail.FromGRPC(err)
+ }
+ return resp.GetAppInfo(), nil
+}
+
+func (c *clientApplicationServiceClient) ReissueAppCert(ctx context.Context, appInfo *vnetv1.AppInfo, targetPort uint16) ([]byte, error) {
+ resp, err := c.clt.ReissueAppCert(ctx, &vnetv1.ReissueAppCertRequest{
+ AppInfo: appInfo,
+ TargetPort: uint32(targetPort),
+ })
+ if err != nil {
+ return nil, trail.FromGRPC(err)
+ }
+ return resp.GetCert(), nil
+}
+
+func (c *clientApplicationServiceClient) SignForApp(ctx context.Context, req *vnetv1.SignForAppRequest) ([]byte, error) {
+ resp, err := c.clt.SignForApp(ctx, req)
+ if err != nil {
+ return nil, trail.FromGRPC(err)
+ }
+ return resp.GetSignature(), nil
+}
+
+func (c *clientApplicationServiceClient) OnNewConnection(ctx context.Context, appKey *vnetv1.AppKey) error {
+ _, err := c.clt.OnNewConnection(ctx, &vnetv1.OnNewConnectionRequest{
+ AppKey: appKey,
+ })
+ if err != nil {
+ return trail.FromGRPC(err)
+ }
+ return nil
+}
+
+func (c *clientApplicationServiceClient) OnInvalidLocalPort(ctx context.Context, appInfo *vnetv1.AppInfo, targetPort uint16) error {
+ _, err := c.clt.OnInvalidLocalPort(ctx, &vnetv1.OnInvalidLocalPortRequest{
+ AppInfo: appInfo,
+ TargetPort: uint32(targetPort),
+ })
+ if err != nil {
+ return trail.FromGRPC(err)
+ }
+ return nil
+}
diff --git a/lib/vnet/client_application_service_other.go b/lib/vnet/client_application_service_other.go
new file mode 100644
index 0000000000000..bff8280aa251f
--- /dev/null
+++ b/lib/vnet/client_application_service_other.go
@@ -0,0 +1,31 @@
+// Teleport
+// Copyright (C) 2024 Gravitational, Inc.
+//
+// This program is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Affero General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// This program is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Affero General Public License for more details.
+//
+// You should have received a copy of the GNU Affero General Public License
+// along with this program. If not, see .
+
+//go:build !windows
+// +build !windows
+
+package vnet
+
+import (
+ "context"
+
+ vnetv1 "github.com/gravitational/teleport/gen/proto/go/teleport/lib/vnet/v1"
+)
+
+func platformAuthenticateProcess(ctx context.Context, req *vnetv1.AuthenticateProcessRequest) error {
+ // This is only called in tests, the real method is only used on Windows.
+ return nil
+}
diff --git a/lib/vnet/client_application_service_windows.go b/lib/vnet/client_application_service_windows.go
new file mode 100644
index 0000000000000..97236228f73d3
--- /dev/null
+++ b/lib/vnet/client_application_service_windows.go
@@ -0,0 +1,33 @@
+// Teleport
+// Copyright (C) 2024 Gravitational, Inc.
+//
+// This program is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Affero General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// This program is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Affero General Public License for more details.
+//
+// You should have received a copy of the GNU Affero General Public License
+// along with this program. If not, see .
+
+package vnet
+
+import (
+ "context"
+
+ "github.com/gravitational/trace"
+
+ vnetv1 "github.com/gravitational/teleport/gen/proto/go/teleport/lib/vnet/v1"
+)
+
+func platformAuthenticateProcess(ctx context.Context, req *vnetv1.AuthenticateProcessRequest) error {
+ if err := connectToPipe(req.GetPipePath()); err != nil {
+ return trace.Wrap(err, "connecting to named pipe")
+ }
+ log.DebugContext(ctx, "Connected to named pipe")
+ return nil
+}
diff --git a/lib/vnet/dns/osnameservers_other.go b/lib/vnet/dns/osnameservers_other.go
index a353020fa0b67..06cca8d0856d3 100644
--- a/lib/vnet/dns/osnameservers_other.go
+++ b/lib/vnet/dns/osnameservers_other.go
@@ -14,8 +14,8 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see .
-//go:build !darwin
-// +build !darwin
+//go:build !darwin && !windows
+// +build !darwin,!windows
package dns
diff --git a/lib/vnet/dns/osnameservers_windows.go b/lib/vnet/dns/osnameservers_windows.go
new file mode 100644
index 0000000000000..301363d9d63ac
--- /dev/null
+++ b/lib/vnet/dns/osnameservers_windows.go
@@ -0,0 +1,30 @@
+// Teleport
+// Copyright (C) 2024 Gravitational, Inc.
+//
+// This program is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Affero General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// This program is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Affero General Public License for more details.
+//
+// You should have received a copy of the GNU Affero General Public License
+// along with this program. If not, see .
+
+package dns
+
+import "context"
+
+type OSUpstreamNameserverSource struct{}
+
+func NewOSUpstreamNameserverSource() (*OSUpstreamNameserverSource, error) {
+ return &OSUpstreamNameserverSource{}, nil
+}
+
+func (s *OSUpstreamNameserverSource) UpstreamNameservers(ctx context.Context) ([]string, error) {
+ // TODO(nklaassen): implement UpstreamNameservers on windows.
+ return nil, nil
+}
diff --git a/lib/vnet/remote_app_provider.go b/lib/vnet/remote_app_provider.go
new file mode 100644
index 0000000000000..5761bea645542
--- /dev/null
+++ b/lib/vnet/remote_app_provider.go
@@ -0,0 +1,111 @@
+package vnet
+
+import (
+ "context"
+ "crypto"
+ "crypto/tls"
+ "crypto/x509"
+ "io"
+
+ vnetv1 "github.com/gravitational/teleport/gen/proto/go/teleport/lib/vnet/v1"
+ "github.com/gravitational/trace"
+)
+
+// remoteAppProvider implements appProvider when the client application is
+// available over gRPC.
+type remoteAppProvider struct {
+ clt *clientApplicationServiceClient
+}
+
+func newRemoteAppProvider(clt *clientApplicationServiceClient) *remoteAppProvider {
+ return &remoteAppProvider{
+ clt: clt,
+ }
+}
+
+func (p *remoteAppProvider) ResolveAppInfo(ctx context.Context, fqdn string) (*vnetv1.AppInfo, error) {
+ return p.clt.ResolveAppInfo(ctx, fqdn)
+}
+
+func (p *remoteAppProvider) ReissueAppCert(ctx context.Context, appInfo *vnetv1.AppInfo, targetPort uint16) (tls.Certificate, error) {
+ cert, err := p.clt.ReissueAppCert(ctx, appInfo, targetPort)
+ if err != nil {
+ return tls.Certificate{}, trace.Wrap(err, "reissuing certificate for app %s", appInfo.GetAppKey().GetName())
+ }
+ signer, err := p.newAppCertSigner(cert, appInfo.GetAppKey(), targetPort)
+ if err != nil {
+ return tls.Certificate{}, trace.Wrap(err)
+ }
+ tlsCert := tls.Certificate{
+ Certificate: [][]byte{cert},
+ PrivateKey: signer,
+ }
+ return tlsCert, nil
+}
+
+func (p *remoteAppProvider) newAppCertSigner(cert []byte, appKey *vnetv1.AppKey, targetPort uint16) (*rpcAppCertSigner, error) {
+ x509Cert, err := x509.ParseCertificate(cert)
+ if err != nil {
+ return nil, trace.Wrap(err, "parsing x509 certificate")
+ }
+ return &rpcAppCertSigner{
+ clt: p.clt,
+ pub: x509Cert.PublicKey,
+ appKey: appKey,
+ targetPort: targetPort,
+ }, nil
+}
+
+// rpcAppCertSigner implements [crypto.Signer] for app TLS signatures that are
+// issued by the client application over gRPC.
+type rpcAppCertSigner struct {
+ clt *clientApplicationServiceClient
+ pub crypto.PublicKey
+ appKey *vnetv1.AppKey
+ targetPort uint16
+}
+
+// Public implements [crypto.Signer.Public] and returns the public key
+// associated with the signer.
+func (s *rpcAppCertSigner) Public() crypto.PublicKey {
+ return s.pub
+}
+
+// Sign implements [crypto.Signer.Sign] and issues a signature over digest for
+// the associated app.
+func (s *rpcAppCertSigner) Sign(rand io.Reader, digest []byte, opts crypto.SignerOpts) ([]byte, error) {
+ protoHash := vnetv1.Hash_HASH_UNSPECIFIED
+ switch opts.HashFunc() {
+ case 0:
+ protoHash = vnetv1.Hash_HASH_NONE
+ case crypto.SHA256:
+ protoHash = vnetv1.Hash_HASH_SHA256
+ }
+ signature, err := s.clt.SignForApp(context.TODO(), &vnetv1.SignForAppRequest{
+ AppKey: s.appKey,
+ TargetPort: uint32(s.targetPort),
+ Digest: digest,
+ Hash: protoHash,
+ })
+ if err != nil {
+ return nil, trace.Wrap(err)
+ }
+ return signature, nil
+}
+
+func (p *remoteAppProvider) OnNewConnection(ctx context.Context, appKey *vnetv1.AppKey) error {
+ if err := p.clt.OnNewConnection(ctx, appKey); err != nil {
+ return trace.Wrap(err)
+ }
+ return nil
+}
+
+func (p *remoteAppProvider) OnInvalidLocalPort(ctx context.Context, appInfo *vnetv1.AppInfo, targetPort uint16) {
+ if err := p.clt.OnInvalidLocalPort(ctx, appInfo, targetPort); err != nil {
+ log.ErrorContext(ctx, "Could not notify client application about invalid local port",
+ "error", err,
+ "app_name", appInfo.GetAppKey().GetName(),
+ "target_port", targetPort,
+ )
+ }
+}
diff --git a/lib/vnet/service_windows.go b/lib/vnet/service_windows.go
index 1387d6cca4407..890b4a3ded2d1 100644
--- a/lib/vnet/service_windows.go
+++ b/lib/vnet/service_windows.go
@@ -24,6 +24,7 @@ import (
"syscall"
"time"
+ "github.com/alecthomas/kingpin/v2"
"github.com/gravitational/trace"
"golang.org/x/sys/windows"
"golang.org/x/sys/windows/svc"
@@ -40,8 +41,8 @@ const (
// runService is called from the normal user process to run the VNet Windows in
// the background and wait for it to exit. It will terminate the service and
// return immediately if [ctx] is canceled.
-func runService(ctx context.Context) error {
- service, err := startService(ctx)
+func runService(ctx context.Context, cfg *windowsAdminProcessConfig) error {
+ service, err := startService(ctx, cfg)
if err != nil {
return trace.Wrap(err)
}
@@ -69,7 +70,7 @@ func runService(ctx context.Context) error {
}
// startService starts the Windows VNet admin service in the background.
-func startService(ctx context.Context) (*mgr.Service, error) {
+func startService(ctx context.Context, cfg *windowsAdminProcessConfig) (*mgr.Service, error) {
// Avoid [mgr.Connect] because it requests elevated permissions.
scManager, err := windows.OpenSCManager(nil /*machine*/, nil /*database*/, windows.SC_MANAGER_CONNECT)
if err != nil {
@@ -88,7 +89,7 @@ func startService(ctx context.Context) (*mgr.Service, error) {
Name: serviceName,
Handle: serviceHandle,
}
- if err := service.Start(ServiceCommand); err != nil {
+ if err := service.Start(ServiceCommand, "--addr", cfg.clientApplicationServiceAddr); err != nil {
return nil, trace.Wrap(err, "starting Windows service %s", serviceName)
}
return service, nil
@@ -157,7 +158,13 @@ loop:
}
func (s *windowsService) run(ctx context.Context, args []string) error {
- if err := runWindowsAdminProcess(ctx); err != nil {
+ var clientApplicationServiceAddr string
+ app := kingpin.New(serviceName, "Teleport Windows Service")
+ serviceCmd := app.Command("vnet-service", "Start the VNet service.")
+ serviceCmd.Flag("addr", "client application service address").Required().StringVar(&clientApplicationServiceAddr)
+ if err := runWindowsAdminProcess(ctx, &windowsAdminProcessConfig{
+ clientApplicationServiceAddr: clientApplicationServiceAddr,
+ }); err != nil {
return trace.Wrap(err, "running admin process")
}
return nil
diff --git a/lib/vnet/user_process_windows.go b/lib/vnet/user_process_windows.go
index 9fccd0bb528b3..975a2e333a842 100644
--- a/lib/vnet/user_process_windows.go
+++ b/lib/vnet/user_process_windows.go
@@ -18,8 +18,15 @@ package vnet
import (
"context"
+ "net"
"github.com/gravitational/trace"
+ "github.com/jonboulle/clockwork"
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/credentials/insecure"
+
+ "github.com/gravitational/teleport/api/utils/grpc/interceptors"
+ vnetv1 "github.com/gravitational/teleport/gen/proto/go/teleport/lib/vnet/v1"
)
// runPlatformUserProcess launches a Windows service in the background that will
@@ -35,10 +42,35 @@ func runPlatformUserProcess(ctx context.Context, config *UserProcessConfig) (pm
}
}()
+ listener, err := net.Listen("tcp", ":0")
+ if err != nil {
+ return nil, trace.Wrap(err, "listening on tcp socket")
+ }
pm, processCtx := newProcessManager()
- pm.AddCriticalBackgroundTask("VNet Windows service", func() error {
- return trace.Wrap(runService(processCtx), "running VNet Windows service in the background")
+ pm.AddCriticalBackgroundTask("tcp socket closer", func() error {
+ <-processCtx.Done()
+ return trace.Wrap(listener.Close())
+ })
+ pm.AddCriticalBackgroundTask("admin process", func() error {
+ return trace.Wrap(runService(processCtx, &windowsAdminProcessConfig{
+ clientApplicationServiceAddr: listener.Addr().String(),
+ }))
+ })
+ pm.AddCriticalBackgroundTask("gRPC service", func() error {
+ log.InfoContext(processCtx, "Starting gRPC service", "addr", listener.Addr().String())
+ grpcServer := grpc.NewServer(
+ grpc.Creds(insecure.NewCredentials()),
+ grpc.UnaryInterceptor(interceptors.GRPCServerUnaryErrorInterceptor),
+ grpc.StreamInterceptor(interceptors.GRPCServerStreamErrorInterceptor),
+ )
+ clock := clockwork.NewRealClock()
+ appProvider := newLocalAppProvider(config.ClientApplication, clock)
+ svc := newClientApplicationService(appProvider)
+ vnetv1.RegisterClientApplicationServiceServer(grpcServer, svc)
+ if err := grpcServer.Serve(listener); err != nil {
+ return trace.Wrap(err, "serving VNet user process gRPC service")
+ }
+ return nil
})
- // TODO(nklaassen): run user process gRPC service.
return pm, nil
}
diff --git a/lib/vnet/vnet_test.go b/lib/vnet/vnet_test.go
index 72dbc0c773909..f6317161d8bfa 100644
--- a/lib/vnet/vnet_test.go
+++ b/lib/vnet/vnet_test.go
@@ -43,6 +43,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"golang.org/x/exp/maps"
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/credentials/insecure"
"gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/adapters/gonet"
"gvisor.dev/gvisor/pkg/tcpip/link/channel"
@@ -53,6 +55,7 @@ import (
headerv1 "github.com/gravitational/teleport/api/gen/proto/go/teleport/header/v1"
"github.com/gravitational/teleport/api/gen/proto/go/teleport/vnet/v1"
"github.com/gravitational/teleport/api/types"
+ "github.com/gravitational/teleport/api/utils/grpc/interceptors"
vnetv1 "github.com/gravitational/teleport/gen/proto/go/teleport/lib/vnet/v1"
"github.com/gravitational/teleport/lib/auth/authclient"
"github.com/gravitational/teleport/lib/cryptosuites"
@@ -841,6 +844,83 @@ func TestOnNewConnection(t *testing.T) {
require.Equal(t, uint32(1), clientApp.onNewConnectionCallCount.Load())
}
+// TestRemoteAppProvider tests basic VNet functionality when remoteAppProvider
+// is used to provider access to the client application over gRPC.
+func TestRemoteAppProvider(t *testing.T) {
+ t.Parallel()
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+
+ clock := clockwork.NewFakeClockAt(time.Now())
+ ca := newSelfSignedCA(t)
+ dialOpts := mustStartFakeWebProxy(ctx, t, ca, clock)
+
+ const appCertLifetime = time.Hour
+ reissueClientCert := func() tls.Certificate {
+ return newClientCert(t, ca, "testclient", clock.Now().Add(appCertLifetime))
+ }
+
+ clientApp := newFakeClientApp(map[string]testClusterSpec{
+ "root.example.com": {
+ apps: []appSpec{
+ appSpec{publicAddr: "echo"},
+ },
+ cidrRange: "192.168.2.0/24",
+ leafClusters: map[string]testClusterSpec{
+ "leaf.example.com": {
+ apps: []appSpec{
+ appSpec{publicAddr: "echo"},
+ },
+ cidrRange: "192.168.2.0/24",
+ },
+ },
+ },
+ }, dialOpts, reissueClientCert, clock)
+
+ grpcServer := grpc.NewServer(
+ grpc.Creds(insecure.NewCredentials()),
+ grpc.UnaryInterceptor(interceptors.GRPCServerUnaryErrorInterceptor),
+ grpc.StreamInterceptor(interceptors.GRPCServerStreamErrorInterceptor),
+ )
+ appProvider := newLocalAppProvider(clientApp, clock)
+ svc := newClientApplicationService(appProvider)
+ vnetv1.RegisterClientApplicationServiceServer(grpcServer, svc)
+ listener, err := net.Listen("tcp", ":0")
+ require.NoError(t, err)
+ utils.RunTestBackgroundTask(ctx, t, &utils.TestBackgroundTask{
+ Name: "user process gRPC server",
+ Task: func(ctx context.Context) error {
+ return trace.Wrap(grpcServer.Serve(listener), "serving VNet user process gRPC service")
+ },
+ Terminate: func() error {
+ grpcServer.Stop()
+ return nil
+ },
+ })
+
+ clt, err := newClientApplicationServiceClient(ctx, listener.Addr().String())
+ require.NoError(t, err)
+ remoteAppProvider := newRemoteAppProvider(clt)
+
+ p := newTestPack(t, ctx, testPackConfig{
+ clock: clock,
+ appProvider: remoteAppProvider,
+ })
+
+ dialCtx, cancel := context.WithTimeout(ctx, 100*time.Millisecond)
+ defer cancel()
+ for _, app := range []string{
+ "echo.root.example.com",
+ "echo.leaf.example.com",
+ } {
+ conn, err := p.dialHost(dialCtx, app, 123)
+ require.NoError(t, err)
+ testEchoConnection(t, conn)
+ }
+ _, err = p.dialHost(dialCtx, "badapp.root.example.com", 123)
+ require.Error(t, err)
+}
+
func randomULAAddress() (tcpip.Address, error) {
var bytes [16]byte
bytes[0] = 0xfd