File: kerberos.go

package info (click to toggle)
golang-github-colinmarc-hdfs 2.3.0-2
  • links: PTS, VCS
  • area: main
  • in suites: bookworm, forky, sid, trixie
  • size: 3,760 kB
  • sloc: sh: 130; xml: 40; makefile: 31
file content (197 lines) | stat: -rw-r--r-- 5,198 bytes parent folder | download
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
package rpc

import (
	"errors"
	"fmt"
	"net"
	"regexp"
	"sort"

	hadoop "github.com/colinmarc/hdfs/v2/internal/protocol/hadoop_common"
	"github.com/colinmarc/hdfs/v2/internal/sasl"
	"github.com/jcmturner/gokrb5/v8/gssapi"
	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
	"github.com/jcmturner/gokrb5/v8/spnego"
	krbtypes "github.com/jcmturner/gokrb5/v8/types"
)

const saslRpcCallId = -33

var (
	errKerberosNotSupported = errors.New("kerberos authentication not supported by namenode")
	krbSPNHost              = regexp.MustCompile(`\A[^/]+/(_HOST)([@/]|\z)`)
)

func (c *NamenodeConnection) doKerberosHandshake() error {
	// Start negotiation, and get the list of supported mechanisms in reply.
	err := c.writeSaslRequest(&hadoop.RpcSaslProto{
		State: hadoop.RpcSaslProto_NEGOTIATE.Enum(),
	})
	if err != nil {
		return err
	}

	resp, err := c.readSaslResponse(hadoop.RpcSaslProto_NEGOTIATE)
	if err != nil {
		return err
	}

	var krbAuth, tokenAuth *hadoop.RpcSaslProto_SaslAuth
	for _, m := range resp.GetAuths() {
		switch *m.Method {
		case "KERBEROS":
			krbAuth = m
		case "TOKEN":
			tokenAuth = m
		default:
		}
	}

	if krbAuth == nil {
		return errKerberosNotSupported
	}

	// Get a ticket from Kerberos, and send the initial token to the namenode.
	token, sessionKey, err := c.getKerberosTicket()
	if err != nil {
		return err
	}

	if tokenAuth != nil {
		challenge, err := sasl.ParseChallenge(tokenAuth.Challenge)
		if err != nil {
			return err
		}

		// Some versions of HDP 3.x expect us to pick the highest Qop, and
		// return a malformed response otherwise.
		sort.Sort(challenge.Qop)
		qop := challenge.Qop[0]

		switch qop {
		case sasl.QopPrivacy, sasl.QopIntegrity:
			// Switch to SASL RPC handler
			c.transport = &saslTransport{
				basicTransport: basicTransport{
					clientID: c.ClientID,
				},
				sessionKey: sessionKey,
				privacy:    qop == sasl.QopPrivacy,
			}
		case sasl.QopAuthentication:
			// No special transport is required.
		default:
			return errors.New("unexpected QOP in challenge")
		}
	}

	err = c.writeSaslRequest(&hadoop.RpcSaslProto{
		State: hadoop.RpcSaslProto_INITIATE.Enum(),
		Token: token.MechTokenBytes,
		Auths: []*hadoop.RpcSaslProto_SaslAuth{krbAuth},
	})
	if err != nil {
		return err
	}

	// In response, we get a server token to verify.
	resp, err = c.readSaslResponse(hadoop.RpcSaslProto_CHALLENGE)
	if err != nil {
		return err
	}

	var nnToken gssapi.WrapToken
	err = nnToken.Unmarshal(resp.GetToken(), true)
	if err != nil {
		return err
	}

	_, err = nnToken.Verify(sessionKey, keyusage.GSSAPI_ACCEPTOR_SEAL)
	if err != nil {
		return fmt.Errorf("invalid server token: %s", err)
	}

	// Sign the payload and send it back to the namenode.
	// TODO: Make sure we can support what is required based on what's in the
	// payload.
	signed, err := gssapi.NewInitiatorWrapToken(nnToken.Payload, sessionKey)
	if err != nil {
		return err
	}

	signedBytes, err := signed.Marshal()
	if err != nil {
		return err
	}

	err = c.writeSaslRequest(&hadoop.RpcSaslProto{
		State: hadoop.RpcSaslProto_RESPONSE.Enum(),
		Token: signedBytes,
	})
	if err != nil {
		return err
	}

	// Read the final response. If it's a SUCCESS, then we're done here.
	_, err = c.readSaslResponse(hadoop.RpcSaslProto_SUCCESS)
	return err
}

func (c *NamenodeConnection) writeSaslRequest(req *hadoop.RpcSaslProto) error {
	rrh := newRPCRequestHeader(saslRpcCallId, c.ClientID)
	packet, err := makeRPCPacket(rrh, req)
	if err != nil {
		return err
	}

	_, err = c.conn.Write(packet)
	return err
}

func (c *NamenodeConnection) readSaslResponse(expectedState hadoop.RpcSaslProto_SaslState) (*hadoop.RpcSaslProto, error) {
	rrh := &hadoop.RpcResponseHeaderProto{}
	resp := &hadoop.RpcSaslProto{}
	err := readRPCPacket(c.conn, rrh, resp)
	if err != nil {
		return nil, err
	} else if int32(rrh.GetCallId()) != saslRpcCallId {
		return nil, errors.New("unexpected sequence number")
	} else if rrh.GetStatus() != hadoop.RpcResponseHeaderProto_SUCCESS {
		return nil, &NamenodeError{
			method:    "sasl",
			message:   rrh.GetErrorMsg(),
			code:      int(rrh.GetErrorDetail()),
			exception: rrh.GetExceptionClassName(),
		}
	} else if resp.GetState() != expectedState {
		return nil, fmt.Errorf("unexpected SASL state: %s", resp.GetState().String())
	}

	return resp, nil
}

// getKerberosTicket returns an initial kerberos negotiation token and the
// paired session key, along with an error if any occured.
func (c *NamenodeConnection) getKerberosTicket() (spnego.NegTokenInit, krbtypes.EncryptionKey, error) {
	host, _, _ := net.SplitHostPort(c.host.address)
	spn := replaceSPNHostWildcard(c.kerberosServicePrincipleName, host)

	ticket, key, err := c.kerberosClient.GetServiceTicket(spn)
	if err != nil {
		return spnego.NegTokenInit{}, key, err
	}

	token, err := spnego.NewNegTokenInitKRB5(c.kerberosClient, ticket, key)
	return token, key, err
}

// replaceSPNHostWildcard substitutes the special string '_HOST' in the given
// SPN for the given (current) host.
func replaceSPNHostWildcard(spn, host string) string {
	res := krbSPNHost.FindStringSubmatchIndex(spn)
	if res == nil || res[2] == -1 {
		return spn
	}

	return spn[:res[2]] + host + spn[res[3]:]
}