forked from colinmarc/hdfs
/
namenode.go
363 lines (309 loc) · 10.1 KB
/
namenode.go
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
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
package rpc
import (
"encoding/binary"
"errors"
"fmt"
"io"
"net"
"sync"
"time"
hadoop "github.com/colinmarc/hdfs/protocol/hadoop_common"
"github.com/golang/protobuf/proto"
)
const (
rpcVersion = 0x09
serviceClass = 0x0
authProtocol = 0x0
protocolClass = "org.apache.hadoop.hdfs.protocol.ClientProtocol"
protocolClassVersion = 1
handshakeCallID = -3
standbyExceptionClass = "org.apache.hadoop.ipc.StandbyException"
)
const backoffDuration = time.Second * 5
// NamenodeConnection represents an open connection to a namenode.
type NamenodeConnection struct {
clientId []byte
clientName string
currentRequestID int
user string
conn net.Conn
host *namenodeHost
hostList []*namenodeHost
reqLock sync.Mutex
}
// NamenodeConnectionOptions represents the configurable options available
// for a NamenodeConnection.
type NamenodeConnectionOptions struct {
Addresses []string
User string
}
// NamenodeError represents an interepreted error from the Namenode, including
// the error code and the java backtrace.
type NamenodeError struct {
Method string
Message string
Code int
Exception string
}
// Desc returns the long form of the error code, as defined in the
// RpcErrorCodeProto in RpcHeader.proto
func (err *NamenodeError) Desc() string {
return hadoop.RpcResponseHeaderProto_RpcErrorCodeProto_name[int32(err.Code)]
}
func (err *NamenodeError) Error() string {
s := fmt.Sprintf("%s call failed with %s", err.Method, err.Desc())
if err.Exception != "" {
s += fmt.Sprintf(" (%s)", err.Exception)
}
return s
}
type namenodeHost struct {
address string
lastError error
lastErrorAt time.Time
}
// NewNamenodeConnection creates a new connection to a namenode and performs an
// initial handshake.
//
// You probably want to use hdfs.New instead, which provides a higher-level
// interface.
func NewNamenodeConnection(address string, user string) (*NamenodeConnection, error) {
return NewNamenodeConnectionWithOptions(NamenodeConnectionOptions{
Addresses: []string{address},
User: user,
})
}
// NewNamenodeConnectionWithOptions creates a new connection to a namenode with
// the given options and performs an initial handshake.
func NewNamenodeConnectionWithOptions(options NamenodeConnectionOptions) (*NamenodeConnection, error) {
// Build the list of hosts to be used for failover.
hostList := make([]*namenodeHost, len(options.Addresses))
for i, addr := range options.Addresses {
hostList[i] = &namenodeHost{address: addr}
}
// The ClientID is reused here both in the RPC headers (which requires a
// "globally unique" ID) and as the "client name" in various requests.
clientId := newClientID()
c := &NamenodeConnection{
clientId: clientId,
clientName: "go-hdfs-" + string(clientId),
user: options.User,
hostList: hostList,
}
err := c.resolveConnection()
if err != nil {
return nil, err
}
return c, nil
}
// WrapNamenodeConnection wraps an existing net.Conn to a Namenode, and preforms
// an initial handshake.
//
// Deprecated: use the higher-level hdfs.New or NewNamenodeConnection instead.
func WrapNamenodeConnection(conn net.Conn, user string) (*NamenodeConnection, error) {
// The ClientID is reused here both in the RPC headers (which requires a
// "globally unique" ID) and as the "client name" in various requests.
clientId := newClientID()
c := &NamenodeConnection{
clientId: clientId,
clientName: "go-hdfs-" + string(clientId),
user: user,
conn: conn,
host: &namenodeHost{},
hostList: make([]*namenodeHost, 0),
}
err := c.writeNamenodeHandshake()
if err != nil {
conn.Close()
return nil, fmt.Errorf("Error performing handshake: %s", err)
}
return c, nil
}
func (c *NamenodeConnection) resolveConnection() error {
if c.conn != nil {
return nil
}
var err error
if c.host != nil {
err = c.host.lastError
}
for _, host := range c.hostList {
if host.lastErrorAt.After(time.Now().Add(-backoffDuration)) {
continue
}
c.host = host
c.conn, err = net.DialTimeout("tcp", host.address, connectTimeout)
if err != nil {
c.markFailure(err)
continue
}
err = c.writeNamenodeHandshake()
if err != nil {
c.markFailure(err)
continue
}
break
}
if c.conn == nil {
return fmt.Errorf("no available namenodes: %s", err)
}
return nil
}
func (c *NamenodeConnection) markFailure(err error) {
if c.conn != nil {
c.conn.Close()
c.conn = nil
}
c.host.lastError = err
c.host.lastErrorAt = time.Now()
}
// ClientName provides a unique identifier for this client, which is required
// for various RPC calls. Confusingly, it's separate from clientID, which is
// used in the RPC header; to make things simpler, it reuses the random bytes
// from that, but adds a prefix to make it human-readable.
func (c *NamenodeConnection) ClientName() string {
return c.clientName
}
// Execute performs an rpc call. It does this by sending req over the wire and
// unmarshaling the result into resp.
func (c *NamenodeConnection) Execute(method string, req proto.Message, resp proto.Message) error {
c.reqLock.Lock()
defer c.reqLock.Unlock()
c.currentRequestID++
for {
err := c.resolveConnection()
if err != nil {
return err
}
err = c.writeRequest(method, req)
if err != nil {
c.markFailure(err)
continue
}
err = c.readResponse(method, resp)
if err != nil {
// Only retry on a standby exception.
if nerr, ok := err.(*NamenodeError); ok && nerr.Exception == standbyExceptionClass {
c.markFailure(err)
continue
}
return err
}
break
}
return nil
}
// RPC definitions
// A request packet:
// +-----------------------------------------------------------+
// | uint32 length of the next three parts |
// +-----------------------------------------------------------+
// | varint length + RpcRequestHeaderProto |
// +-----------------------------------------------------------+
// | varint length + RequestHeaderProto |
// +-----------------------------------------------------------+
// | varint length + Request |
// +-----------------------------------------------------------+
func (c *NamenodeConnection) writeRequest(method string, req proto.Message) error {
rrh := newRPCRequestHeader(c.currentRequestID, c.clientId)
rh := newRequestHeader(method)
reqBytes, err := makeRPCPacket(rrh, rh, req)
if err != nil {
return err
}
_, err = c.conn.Write(reqBytes)
return err
}
// A response from the namenode:
// +-----------------------------------------------------------+
// | uint32 length of the next two parts |
// +-----------------------------------------------------------+
// | varint length + RpcResponseHeaderProto |
// +-----------------------------------------------------------+
// | varint length + Response |
// +-----------------------------------------------------------+
func (c *NamenodeConnection) readResponse(method string, resp proto.Message) error {
var packetLength uint32
err := binary.Read(c.conn, binary.BigEndian, &packetLength)
if err != nil {
return err
}
packet := make([]byte, packetLength)
_, err = io.ReadFull(c.conn, packet)
if err != nil {
return err
}
rrh := &hadoop.RpcResponseHeaderProto{}
err = readRPCPacket(packet, rrh, resp)
if rrh.GetStatus() != hadoop.RpcResponseHeaderProto_SUCCESS {
return &NamenodeError{
Method: method,
Message: rrh.GetErrorMsg(),
Code: int(rrh.GetErrorDetail()),
Exception: rrh.GetExceptionClassName(),
}
} else if int(rrh.GetCallId()) != c.currentRequestID {
return errors.New("Error reading response: unexpected sequence number")
}
return nil
}
// A handshake packet:
// +-----------------------------------------------------------+
// | Header, 4 bytes ("hrpc") |
// +-----------------------------------------------------------+
// | Version, 1 byte (default verion 0x09) |
// +-----------------------------------------------------------+
// | RPC service class, 1 byte (0x00) |
// +-----------------------------------------------------------+
// | Auth protocol, 1 byte (Auth method None = 0x00) |
// +-----------------------------------------------------------+
// | uint32 length of the next two parts |
// +-----------------------------------------------------------+
// | varint length + RpcRequestHeaderProto |
// +-----------------------------------------------------------+
// | varint length + IpcConnectionContextProto |
// +-----------------------------------------------------------+
func (c *NamenodeConnection) writeNamenodeHandshake() error {
rpcHeader := []byte{
0x68, 0x72, 0x70, 0x63, // "hrpc"
rpcVersion, serviceClass, authProtocol,
}
rrh := newRPCRequestHeader(handshakeCallID, c.clientId)
cc := newConnectionContext(c.user)
packet, err := makeRPCPacket(rrh, cc)
if err != nil {
return err
}
_, err = c.conn.Write(append(rpcHeader, packet...))
return err
}
// Close terminates all underlying socket connections to remote server.
func (c *NamenodeConnection) Close() error {
if c.conn != nil {
return c.conn.Close()
}
return nil
}
func newRPCRequestHeader(id int, clientID []byte) *hadoop.RpcRequestHeaderProto {
return &hadoop.RpcRequestHeaderProto{
RpcKind: hadoop.RpcKindProto_RPC_PROTOCOL_BUFFER.Enum(),
RpcOp: hadoop.RpcRequestHeaderProto_RPC_FINAL_PACKET.Enum(),
CallId: proto.Int32(int32(id)),
ClientId: clientID,
}
}
func newRequestHeader(methodName string) *hadoop.RequestHeaderProto {
return &hadoop.RequestHeaderProto{
MethodName: proto.String(methodName),
DeclaringClassProtocolName: proto.String(protocolClass),
ClientProtocolVersion: proto.Uint64(uint64(protocolClassVersion)),
}
}
func newConnectionContext(user string) *hadoop.IpcConnectionContextProto {
return &hadoop.IpcConnectionContextProto{
UserInfo: &hadoop.UserInformationProto{
EffectiveUser: proto.String(user),
},
Protocol: proto.String(protocolClass),
}
}