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 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163 1164 1165 1166 1167 1168 1169 1170 1171 1172 1173 1174 1175 1176 1177 1178 1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362 1363 1364 1365 1366 1367 1368 1369 1370 1371 1372 1373 1374 1375 1376 1377 1378 1379 1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399 1400 1401 1402 1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420 1421 1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432 1433 1434 1435 1436 1437 1438 1439 1440 1441 1442 1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483 1484 1485 1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499 1500 1501 1502 1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578 1579 1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618 1619 1620 1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651 1652 1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680 1681 1682 1683 1684 1685 1686 1687 1688 1689 1690 1691 1692 1693 1694 1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708 1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735 1736 1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778 1779 1780 1781 1782 1783 1784 1785 1786 1787 1788
|
package sarama
import (
"crypto/tls"
"encoding/binary"
"errors"
"fmt"
"io"
"math/rand"
"net"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/rcrowley/go-metrics"
)
// Broker represents a single Kafka broker connection. All operations on this object are entirely concurrency-safe.
type Broker struct {
conf *Config
rack *string
id int32
addr string
correlationID int32
conn net.Conn
connErr error
lock sync.Mutex
opened int32
responses chan *responsePromise
done chan bool
metricRegistry metrics.Registry
incomingByteRate metrics.Meter
requestRate metrics.Meter
fetchRate metrics.Meter
requestSize metrics.Histogram
requestLatency metrics.Histogram
outgoingByteRate metrics.Meter
responseRate metrics.Meter
responseSize metrics.Histogram
requestsInFlight metrics.Counter
protocolRequestsRate map[int16]metrics.Meter
brokerIncomingByteRate metrics.Meter
brokerRequestRate metrics.Meter
brokerFetchRate metrics.Meter
brokerRequestSize metrics.Histogram
brokerRequestLatency metrics.Histogram
brokerOutgoingByteRate metrics.Meter
brokerResponseRate metrics.Meter
brokerResponseSize metrics.Histogram
brokerRequestsInFlight metrics.Counter
brokerThrottleTime metrics.Histogram
brokerProtocolRequestsRate map[int16]metrics.Meter
kerberosAuthenticator GSSAPIKerberosAuth
clientSessionReauthenticationTimeMs int64
throttleTimer *time.Timer
throttleTimerLock sync.Mutex
}
// SASLMechanism specifies the SASL mechanism the client uses to authenticate with the broker
type SASLMechanism string
const (
// SASLTypeOAuth represents the SASL/OAUTHBEARER mechanism (Kafka 2.0.0+)
SASLTypeOAuth = "OAUTHBEARER"
// SASLTypePlaintext represents the SASL/PLAIN mechanism
SASLTypePlaintext = "PLAIN"
// SASLTypeSCRAMSHA256 represents the SCRAM-SHA-256 mechanism.
SASLTypeSCRAMSHA256 = "SCRAM-SHA-256"
// SASLTypeSCRAMSHA512 represents the SCRAM-SHA-512 mechanism.
SASLTypeSCRAMSHA512 = "SCRAM-SHA-512"
SASLTypeGSSAPI = "GSSAPI"
// SASLHandshakeV0 is v0 of the Kafka SASL handshake protocol. Client and
// server negotiate SASL auth using opaque packets.
SASLHandshakeV0 = int16(0)
// SASLHandshakeV1 is v1 of the Kafka SASL handshake protocol. Client and
// server negotiate SASL by wrapping tokens with Kafka protocol headers.
SASLHandshakeV1 = int16(1)
// SASLExtKeyAuth is the reserved extension key name sent as part of the
// SASL/OAUTHBEARER initial client response
SASLExtKeyAuth = "auth"
)
// AccessToken contains an access token used to authenticate a
// SASL/OAUTHBEARER client along with associated metadata.
type AccessToken struct {
// Token is the access token payload.
Token string
// Extensions is a optional map of arbitrary key-value pairs that can be
// sent with the SASL/OAUTHBEARER initial client response. These values are
// ignored by the SASL server if they are unexpected. This feature is only
// supported by Kafka >= 2.1.0.
Extensions map[string]string
}
// AccessTokenProvider is the interface that encapsulates how implementors
// can generate access tokens for Kafka broker authentication.
type AccessTokenProvider interface {
// Token returns an access token. The implementation should ensure token
// reuse so that multiple calls at connect time do not create multiple
// tokens. The implementation should also periodically refresh the token in
// order to guarantee that each call returns an unexpired token. This
// method should not block indefinitely--a timeout error should be returned
// after a short period of inactivity so that the broker connection logic
// can log debugging information and retry.
Token() (*AccessToken, error)
}
// SCRAMClient is a an interface to a SCRAM
// client implementation.
type SCRAMClient interface {
// Begin prepares the client for the SCRAM exchange
// with the server with a user name and a password
Begin(userName, password, authzID string) error
// Step steps client through the SCRAM exchange. It is
// called repeatedly until it errors or `Done` returns true.
Step(challenge string) (response string, err error)
// Done should return true when the SCRAM conversation
// is over.
Done() bool
}
type responsePromise struct {
requestTime time.Time
correlationID int32
headerVersion int16
handler func([]byte, error)
packets chan []byte
errors chan error
}
func (p *responsePromise) handle(packets []byte, err error) {
// Use callback when provided
if p.handler != nil {
p.handler(packets, err)
return
}
// Otherwise fallback to using channels
if err != nil {
p.errors <- err
return
}
p.packets <- packets
}
// NewBroker creates and returns a Broker targeting the given host:port address.
// This does not attempt to actually connect, you have to call Open() for that.
func NewBroker(addr string) *Broker {
return &Broker{id: -1, addr: addr}
}
// Open tries to connect to the Broker if it is not already connected or connecting, but does not block
// waiting for the connection to complete. This means that any subsequent operations on the broker will
// block waiting for the connection to succeed or fail. To get the effect of a fully synchronous Open call,
// follow it by a call to Connected(). The only errors Open will return directly are ConfigurationError or
// AlreadyConnected. If conf is nil, the result of NewConfig() is used.
func (b *Broker) Open(conf *Config) error {
if !atomic.CompareAndSwapInt32(&b.opened, 0, 1) {
return ErrAlreadyConnected
}
if conf == nil {
conf = NewConfig()
}
err := conf.Validate()
if err != nil {
return err
}
usingApiVersionsRequests := conf.Version.IsAtLeast(V2_4_0_0) && conf.ApiVersionsRequest
b.lock.Lock()
if b.metricRegistry == nil {
b.metricRegistry = newCleanupRegistry(conf.MetricRegistry)
}
go withRecover(func() {
defer func() {
b.lock.Unlock()
// Send an ApiVersionsRequest to identify the client (KIP-511).
// Ideally Sarama would use the response to control protocol versions,
// but for now just fire-and-forget just to send
if usingApiVersionsRequests {
_, err = b.ApiVersions(&ApiVersionsRequest{
Version: 3,
ClientSoftwareName: defaultClientSoftwareName,
ClientSoftwareVersion: version(),
})
if err != nil {
Logger.Printf("Error while sending ApiVersionsRequest to broker %s: %s\n", b.addr, err)
}
}
}()
dialer := conf.getDialer()
b.conn, b.connErr = dialer.Dial("tcp", b.addr)
if b.connErr != nil {
Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, b.connErr)
b.conn = nil
atomic.StoreInt32(&b.opened, 0)
return
}
if conf.Net.TLS.Enable {
b.conn = tls.Client(b.conn, validServerNameTLS(b.addr, conf.Net.TLS.Config))
}
b.conn = newBufConn(b.conn)
b.conf = conf
// Create or reuse the global metrics shared between brokers
b.incomingByteRate = metrics.GetOrRegisterMeter("incoming-byte-rate", b.metricRegistry)
b.requestRate = metrics.GetOrRegisterMeter("request-rate", b.metricRegistry)
b.fetchRate = metrics.GetOrRegisterMeter("consumer-fetch-rate", b.metricRegistry)
b.requestSize = getOrRegisterHistogram("request-size", b.metricRegistry)
b.requestLatency = getOrRegisterHistogram("request-latency-in-ms", b.metricRegistry)
b.outgoingByteRate = metrics.GetOrRegisterMeter("outgoing-byte-rate", b.metricRegistry)
b.responseRate = metrics.GetOrRegisterMeter("response-rate", b.metricRegistry)
b.responseSize = getOrRegisterHistogram("response-size", b.metricRegistry)
b.requestsInFlight = metrics.GetOrRegisterCounter("requests-in-flight", b.metricRegistry)
b.protocolRequestsRate = map[int16]metrics.Meter{}
// Do not gather metrics for seeded broker (only used during bootstrap) because they share
// the same id (-1) and are already exposed through the global metrics above
if b.id >= 0 && !metrics.UseNilMetrics {
b.registerMetrics()
}
if conf.Net.SASL.Mechanism == SASLTypeOAuth && conf.Net.SASL.Version == SASLHandshakeV0 {
conf.Net.SASL.Version = SASLHandshakeV1
}
useSaslV0 := conf.Net.SASL.Version == SASLHandshakeV0 || conf.Net.SASL.Mechanism == SASLTypeGSSAPI
if conf.Net.SASL.Enable && useSaslV0 {
b.connErr = b.authenticateViaSASLv0()
if b.connErr != nil {
err = b.conn.Close()
if err == nil {
DebugLogger.Printf("Closed connection to broker %s due to SASL v0 auth error: %s\n", b.addr, b.connErr)
} else {
Logger.Printf("Error while closing connection to broker %s (due to SASL v0 auth error: %s): %s\n", b.addr, b.connErr, err)
}
b.conn = nil
atomic.StoreInt32(&b.opened, 0)
return
}
}
b.done = make(chan bool)
b.responses = make(chan *responsePromise, b.conf.Net.MaxOpenRequests-1)
go withRecover(b.responseReceiver)
if conf.Net.SASL.Enable && !useSaslV0 {
b.connErr = b.authenticateViaSASLv1()
if b.connErr != nil {
close(b.responses)
<-b.done
err = b.conn.Close()
if err == nil {
DebugLogger.Printf("Closed connection to broker %s due to SASL v1 auth error: %s\n", b.addr, b.connErr)
} else {
Logger.Printf("Error while closing connection to broker %s (due to SASL v1 auth error: %s): %s\n", b.addr, b.connErr, err)
}
b.conn = nil
atomic.StoreInt32(&b.opened, 0)
return
}
}
if b.id >= 0 {
DebugLogger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id)
} else {
DebugLogger.Printf("Connected to broker at %s (unregistered)\n", b.addr)
}
})
return nil
}
func (b *Broker) ResponseSize() int {
b.lock.Lock()
defer b.lock.Unlock()
return len(b.responses)
}
// Connected returns true if the broker is connected and false otherwise. If the broker is not
// connected but it had tried to connect, the error from that connection attempt is also returned.
func (b *Broker) Connected() (bool, error) {
b.lock.Lock()
defer b.lock.Unlock()
return b.conn != nil, b.connErr
}
// TLSConnectionState returns the client's TLS connection state. The second return value is false if this is not a tls connection or the connection has not yet been established.
func (b *Broker) TLSConnectionState() (state tls.ConnectionState, ok bool) {
b.lock.Lock()
defer b.lock.Unlock()
if b.conn == nil {
return state, false
}
conn := b.conn
if bconn, ok := b.conn.(*bufConn); ok {
conn = bconn.Conn
}
if tc, ok := conn.(*tls.Conn); ok {
return tc.ConnectionState(), true
}
return state, false
}
// Close closes the broker resources
func (b *Broker) Close() error {
b.lock.Lock()
defer b.lock.Unlock()
if b.conn == nil {
return ErrNotConnected
}
close(b.responses)
<-b.done
err := b.conn.Close()
b.conn = nil
b.connErr = nil
b.done = nil
b.responses = nil
b.metricRegistry.UnregisterAll()
if err == nil {
DebugLogger.Printf("Closed connection to broker %s\n", b.addr)
} else {
Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err)
}
atomic.StoreInt32(&b.opened, 0)
return err
}
// ID returns the broker ID retrieved from Kafka's metadata, or -1 if that is not known.
func (b *Broker) ID() int32 {
return b.id
}
// Addr returns the broker address as either retrieved from Kafka's metadata or passed to NewBroker.
func (b *Broker) Addr() string {
return b.addr
}
// Rack returns the broker's rack as retrieved from Kafka's metadata or the
// empty string if it is not known. The returned value corresponds to the
// broker's broker.rack configuration setting. Requires protocol version to be
// at least v0.10.0.0.
func (b *Broker) Rack() string {
if b.rack == nil {
return ""
}
return *b.rack
}
// GetMetadata send a metadata request and returns a metadata response or error
func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error) {
response := new(MetadataResponse)
response.Version = request.Version // Required to ensure use of the correct response header version
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// GetConsumerMetadata send a consumer metadata request and returns a consumer metadata response or error
func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) {
response := new(ConsumerMetadataResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// FindCoordinator sends a find coordinate request and returns a response or error
func (b *Broker) FindCoordinator(request *FindCoordinatorRequest) (*FindCoordinatorResponse, error) {
response := new(FindCoordinatorResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// GetAvailableOffsets return an offset response or error
func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) {
response := new(OffsetResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// ProduceCallback function is called once the produce response has been parsed
// or could not be read.
type ProduceCallback func(*ProduceResponse, error)
// AsyncProduce sends a produce request and eventually call the provided callback
// with a produce response or an error.
//
// Waiting for the response is generally not blocking on the contrary to using Produce.
// If the maximum number of in flight request configured is reached then
// the request will be blocked till a previous response is received.
//
// When configured with RequiredAcks == NoResponse, the callback will not be invoked.
// If an error is returned because the request could not be sent then the callback
// will not be invoked either.
//
// Make sure not to Close the broker in the callback as it will lead to a deadlock.
func (b *Broker) AsyncProduce(request *ProduceRequest, cb ProduceCallback) error {
b.lock.Lock()
defer b.lock.Unlock()
needAcks := request.RequiredAcks != NoResponse
// Use a nil promise when no acks is required
var promise *responsePromise
if needAcks {
metricRegistry := b.metricRegistry
// Create ProduceResponse early to provide the header version
res := new(ProduceResponse)
promise = &responsePromise{
headerVersion: res.headerVersion(),
// Packets will be converted to a ProduceResponse in the responseReceiver goroutine
handler: func(packets []byte, err error) {
if err != nil {
// Failed request
cb(nil, err)
return
}
if err := versionedDecode(packets, res, request.version(), metricRegistry); err != nil {
// Malformed response
cb(nil, err)
return
}
// Well-formed response
b.handleThrottledResponse(res)
cb(res, nil)
},
}
}
return b.sendWithPromise(request, promise)
}
// Produce returns a produce response or error
func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
var (
response *ProduceResponse
err error
)
if request.RequiredAcks == NoResponse {
err = b.sendAndReceive(request, nil)
} else {
response = new(ProduceResponse)
err = b.sendAndReceive(request, response)
}
if err != nil {
return nil, err
}
return response, nil
}
// Fetch returns a FetchResponse or error
func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) {
defer func() {
if b.fetchRate != nil {
b.fetchRate.Mark(1)
}
if b.brokerFetchRate != nil {
b.brokerFetchRate.Mark(1)
}
}()
response := new(FetchResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// CommitOffset return an Offset commit response or error
func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) {
response := new(OffsetCommitResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// FetchOffset returns an offset fetch response or error
func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) {
response := new(OffsetFetchResponse)
response.Version = request.Version // needed to handle the two header versions
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// JoinGroup returns a join group response or error
func (b *Broker) JoinGroup(request *JoinGroupRequest) (*JoinGroupResponse, error) {
response := new(JoinGroupResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// SyncGroup returns a sync group response or error
func (b *Broker) SyncGroup(request *SyncGroupRequest) (*SyncGroupResponse, error) {
response := new(SyncGroupResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// LeaveGroup return a leave group response or error
func (b *Broker) LeaveGroup(request *LeaveGroupRequest) (*LeaveGroupResponse, error) {
response := new(LeaveGroupResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// Heartbeat returns a heartbeat response or error
func (b *Broker) Heartbeat(request *HeartbeatRequest) (*HeartbeatResponse, error) {
response := new(HeartbeatResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// ListGroups return a list group response or error
func (b *Broker) ListGroups(request *ListGroupsRequest) (*ListGroupsResponse, error) {
response := new(ListGroupsResponse)
response.Version = request.Version // Required to ensure use of the correct response header version
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// DescribeGroups return describe group response or error
func (b *Broker) DescribeGroups(request *DescribeGroupsRequest) (*DescribeGroupsResponse, error) {
response := new(DescribeGroupsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// ApiVersions return api version response or error
func (b *Broker) ApiVersions(request *ApiVersionsRequest) (*ApiVersionsResponse, error) {
response := new(ApiVersionsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// CreateTopics send a create topic request and returns create topic response
func (b *Broker) CreateTopics(request *CreateTopicsRequest) (*CreateTopicsResponse, error) {
response := new(CreateTopicsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// DeleteTopics sends a delete topic request and returns delete topic response
func (b *Broker) DeleteTopics(request *DeleteTopicsRequest) (*DeleteTopicsResponse, error) {
response := new(DeleteTopicsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// CreatePartitions sends a create partition request and returns create
// partitions response or error
func (b *Broker) CreatePartitions(request *CreatePartitionsRequest) (*CreatePartitionsResponse, error) {
response := new(CreatePartitionsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// AlterPartitionReassignments sends a alter partition reassignments request and
// returns alter partition reassignments response
func (b *Broker) AlterPartitionReassignments(request *AlterPartitionReassignmentsRequest) (*AlterPartitionReassignmentsResponse, error) {
response := new(AlterPartitionReassignmentsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// ListPartitionReassignments sends a list partition reassignments request and
// returns list partition reassignments response
func (b *Broker) ListPartitionReassignments(request *ListPartitionReassignmentsRequest) (*ListPartitionReassignmentsResponse, error) {
response := new(ListPartitionReassignmentsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// ElectLeaders sends aa elect leaders request and returns list partitions elect result
func (b *Broker) ElectLeaders(request *ElectLeadersRequest) (*ElectLeadersResponse, error) {
response := new(ElectLeadersResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// DeleteRecords send a request to delete records and return delete record
// response or error
func (b *Broker) DeleteRecords(request *DeleteRecordsRequest) (*DeleteRecordsResponse, error) {
response := new(DeleteRecordsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// DescribeAcls sends a describe acl request and returns a response or error
func (b *Broker) DescribeAcls(request *DescribeAclsRequest) (*DescribeAclsResponse, error) {
response := new(DescribeAclsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// CreateAcls sends a create acl request and returns a response or error
func (b *Broker) CreateAcls(request *CreateAclsRequest) (*CreateAclsResponse, error) {
response := new(CreateAclsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
errs := make([]error, 0)
for _, res := range response.AclCreationResponses {
if !errors.Is(res.Err, ErrNoError) {
errs = append(errs, res.Err)
}
}
if len(errs) > 0 {
return response, Wrap(ErrCreateACLs, errs...)
}
return response, nil
}
// DeleteAcls sends a delete acl request and returns a response or error
func (b *Broker) DeleteAcls(request *DeleteAclsRequest) (*DeleteAclsResponse, error) {
response := new(DeleteAclsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// InitProducerID sends an init producer request and returns a response or error
func (b *Broker) InitProducerID(request *InitProducerIDRequest) (*InitProducerIDResponse, error) {
response := new(InitProducerIDResponse)
response.Version = request.version()
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// AddPartitionsToTxn send a request to add partition to txn and returns
// a response or error
func (b *Broker) AddPartitionsToTxn(request *AddPartitionsToTxnRequest) (*AddPartitionsToTxnResponse, error) {
response := new(AddPartitionsToTxnResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// AddOffsetsToTxn sends a request to add offsets to txn and returns a response
// or error
func (b *Broker) AddOffsetsToTxn(request *AddOffsetsToTxnRequest) (*AddOffsetsToTxnResponse, error) {
response := new(AddOffsetsToTxnResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// EndTxn sends a request to end txn and returns a response or error
func (b *Broker) EndTxn(request *EndTxnRequest) (*EndTxnResponse, error) {
response := new(EndTxnResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// TxnOffsetCommit sends a request to commit transaction offsets and returns
// a response or error
func (b *Broker) TxnOffsetCommit(request *TxnOffsetCommitRequest) (*TxnOffsetCommitResponse, error) {
response := new(TxnOffsetCommitResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// DescribeConfigs sends a request to describe config and returns a response or
// error
func (b *Broker) DescribeConfigs(request *DescribeConfigsRequest) (*DescribeConfigsResponse, error) {
response := new(DescribeConfigsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// AlterConfigs sends a request to alter config and return a response or error
func (b *Broker) AlterConfigs(request *AlterConfigsRequest) (*AlterConfigsResponse, error) {
response := new(AlterConfigsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// IncrementalAlterConfigs sends a request to incremental alter config and return a response or error
func (b *Broker) IncrementalAlterConfigs(request *IncrementalAlterConfigsRequest) (*IncrementalAlterConfigsResponse, error) {
response := new(IncrementalAlterConfigsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// DeleteGroups sends a request to delete groups and returns a response or error
func (b *Broker) DeleteGroups(request *DeleteGroupsRequest) (*DeleteGroupsResponse, error) {
response := new(DeleteGroupsResponse)
if err := b.sendAndReceive(request, response); err != nil {
return nil, err
}
return response, nil
}
// DeleteOffsets sends a request to delete group offsets and returns a response or error
func (b *Broker) DeleteOffsets(request *DeleteOffsetsRequest) (*DeleteOffsetsResponse, error) {
response := new(DeleteOffsetsResponse)
if err := b.sendAndReceive(request, response); err != nil {
return nil, err
}
return response, nil
}
// DescribeLogDirs sends a request to get the broker's log dir paths and sizes
func (b *Broker) DescribeLogDirs(request *DescribeLogDirsRequest) (*DescribeLogDirsResponse, error) {
response := new(DescribeLogDirsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// DescribeUserScramCredentials sends a request to get SCRAM users
func (b *Broker) DescribeUserScramCredentials(req *DescribeUserScramCredentialsRequest) (*DescribeUserScramCredentialsResponse, error) {
res := new(DescribeUserScramCredentialsResponse)
err := b.sendAndReceive(req, res)
if err != nil {
return nil, err
}
return res, err
}
func (b *Broker) AlterUserScramCredentials(req *AlterUserScramCredentialsRequest) (*AlterUserScramCredentialsResponse, error) {
res := new(AlterUserScramCredentialsResponse)
err := b.sendAndReceive(req, res)
if err != nil {
return nil, err
}
return res, nil
}
// DescribeClientQuotas sends a request to get the broker's quotas
func (b *Broker) DescribeClientQuotas(request *DescribeClientQuotasRequest) (*DescribeClientQuotasResponse, error) {
response := new(DescribeClientQuotasResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// AlterClientQuotas sends a request to alter the broker's quotas
func (b *Broker) AlterClientQuotas(request *AlterClientQuotasRequest) (*AlterClientQuotasResponse, error) {
response := new(AlterClientQuotasResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// readFull ensures the conn ReadDeadline has been setup before making a
// call to io.ReadFull
func (b *Broker) readFull(buf []byte) (n int, err error) {
if err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout)); err != nil {
return 0, err
}
return io.ReadFull(b.conn, buf)
}
// write ensures the conn WriteDeadline has been setup before making a
// call to conn.Write
func (b *Broker) write(buf []byte) (n int, err error) {
if err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout)); err != nil {
return 0, err
}
return b.conn.Write(buf)
}
// b.lock must be held by caller
func (b *Broker) send(rb protocolBody, promiseResponse bool, responseHeaderVersion int16) (*responsePromise, error) {
var promise *responsePromise
if promiseResponse {
// Packets or error will be sent to the following channels
// once the response is received
promise = makeResponsePromise(responseHeaderVersion)
}
if err := b.sendWithPromise(rb, promise); err != nil {
return nil, err
}
return promise, nil
}
func makeResponsePromise(responseHeaderVersion int16) *responsePromise {
promise := &responsePromise{
headerVersion: responseHeaderVersion,
packets: make(chan []byte),
errors: make(chan error),
}
return promise
}
// b.lock must be held by caller
func (b *Broker) sendWithPromise(rb protocolBody, promise *responsePromise) error {
if b.conn == nil {
if b.connErr != nil {
return b.connErr
}
return ErrNotConnected
}
if b.clientSessionReauthenticationTimeMs > 0 && currentUnixMilli() > b.clientSessionReauthenticationTimeMs {
err := b.authenticateViaSASLv1()
if err != nil {
return err
}
}
return b.sendInternal(rb, promise)
}
// b.lock must be held by caller
func (b *Broker) sendInternal(rb protocolBody, promise *responsePromise) error {
if !b.conf.Version.IsAtLeast(rb.requiredVersion()) {
return ErrUnsupportedVersion
}
req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
buf, err := encode(req, b.metricRegistry)
if err != nil {
return err
}
// check and wait if throttled
b.waitIfThrottled()
requestTime := time.Now()
// Will be decremented in responseReceiver (except error or request with NoResponse)
b.addRequestInFlightMetrics(1)
bytes, err := b.write(buf)
b.updateOutgoingCommunicationMetrics(bytes)
b.updateProtocolMetrics(rb)
if err != nil {
b.addRequestInFlightMetrics(-1)
return err
}
b.correlationID++
if promise == nil {
// Record request latency without the response
b.updateRequestLatencyAndInFlightMetrics(time.Since(requestTime))
return nil
}
promise.requestTime = requestTime
promise.correlationID = req.correlationID
b.responses <- promise
return nil
}
func (b *Broker) sendAndReceive(req protocolBody, res protocolBody) error {
b.lock.Lock()
defer b.lock.Unlock()
responseHeaderVersion := int16(-1)
if res != nil {
responseHeaderVersion = res.headerVersion()
}
promise, err := b.send(req, res != nil, responseHeaderVersion)
if err != nil {
return err
}
if promise == nil {
return nil
}
err = handleResponsePromise(req, res, promise, b.metricRegistry)
if err != nil {
return err
}
if res != nil {
b.handleThrottledResponse(res)
}
return nil
}
func handleResponsePromise(req protocolBody, res protocolBody, promise *responsePromise, metricRegistry metrics.Registry) error {
select {
case buf := <-promise.packets:
return versionedDecode(buf, res, req.version(), metricRegistry)
case err := <-promise.errors:
return err
}
}
func (b *Broker) decode(pd packetDecoder, version int16) (err error) {
b.id, err = pd.getInt32()
if err != nil {
return err
}
var host string
if version < 9 {
host, err = pd.getString()
} else {
host, err = pd.getCompactString()
}
if err != nil {
return err
}
port, err := pd.getInt32()
if err != nil {
return err
}
if version >= 1 && version < 9 {
b.rack, err = pd.getNullableString()
} else if version >= 9 {
b.rack, err = pd.getCompactNullableString()
}
if err != nil {
return err
}
b.addr = net.JoinHostPort(host, fmt.Sprint(port))
if _, _, err := net.SplitHostPort(b.addr); err != nil {
return err
}
if version >= 9 {
_, err := pd.getEmptyTaggedFieldArray()
if err != nil {
return err
}
}
return nil
}
func (b *Broker) encode(pe packetEncoder, version int16) (err error) {
host, portstr, err := net.SplitHostPort(b.addr)
if err != nil {
return err
}
port, err := strconv.ParseInt(portstr, 10, 32)
if err != nil {
return err
}
pe.putInt32(b.id)
if version < 9 {
err = pe.putString(host)
} else {
err = pe.putCompactString(host)
}
if err != nil {
return err
}
pe.putInt32(int32(port))
if version >= 1 {
if version < 9 {
err = pe.putNullableString(b.rack)
} else {
err = pe.putNullableCompactString(b.rack)
}
if err != nil {
return err
}
}
if version >= 9 {
pe.putEmptyTaggedFieldArray()
}
return nil
}
func (b *Broker) responseReceiver() {
var dead error
for response := range b.responses {
if dead != nil {
// This was previously incremented in send() and
// we are not calling updateIncomingCommunicationMetrics()
b.addRequestInFlightMetrics(-1)
response.handle(nil, dead)
continue
}
headerLength := getHeaderLength(response.headerVersion)
header := make([]byte, headerLength)
bytesReadHeader, err := b.readFull(header)
requestLatency := time.Since(response.requestTime)
if err != nil {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
dead = err
response.handle(nil, err)
continue
}
decodedHeader := responseHeader{}
err = versionedDecode(header, &decodedHeader, response.headerVersion, b.metricRegistry)
if err != nil {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
dead = err
response.handle(nil, err)
continue
}
if decodedHeader.correlationID != response.correlationID {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
// TODO if decoded ID < cur ID, discard until we catch up
// TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response
dead = PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)}
response.handle(nil, dead)
continue
}
buf := make([]byte, decodedHeader.length-int32(headerLength)+4)
bytesReadBody, err := b.readFull(buf)
b.updateIncomingCommunicationMetrics(bytesReadHeader+bytesReadBody, requestLatency)
if err != nil {
dead = err
response.handle(nil, err)
continue
}
response.handle(buf, nil)
}
close(b.done)
}
func getHeaderLength(headerVersion int16) int8 {
if headerVersion < 1 {
return 8
} else {
// header contains additional tagged field length (0), we don't support actual tags yet.
return 9
}
}
func (b *Broker) authenticateViaSASLv0() error {
switch b.conf.Net.SASL.Mechanism {
case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512:
return b.sendAndReceiveSASLSCRAMv0()
case SASLTypeGSSAPI:
return b.sendAndReceiveKerberos()
default:
return b.sendAndReceiveSASLPlainAuthV0()
}
}
func (b *Broker) authenticateViaSASLv1() error {
metricRegistry := b.metricRegistry
if b.conf.Net.SASL.Handshake {
handshakeRequest := &SaslHandshakeRequest{Mechanism: string(b.conf.Net.SASL.Mechanism), Version: b.conf.Net.SASL.Version}
handshakeResponse := new(SaslHandshakeResponse)
prom := makeResponsePromise(handshakeResponse.version())
handshakeErr := b.sendInternal(handshakeRequest, prom)
if handshakeErr != nil {
Logger.Printf("Error while performing SASL handshake %s: %s\n", b.addr, handshakeErr)
return handshakeErr
}
handshakeErr = handleResponsePromise(handshakeRequest, handshakeResponse, prom, metricRegistry)
if handshakeErr != nil {
Logger.Printf("Error while handling SASL handshake response %s: %s\n", b.addr, handshakeErr)
return handshakeErr
}
if !errors.Is(handshakeResponse.Err, ErrNoError) {
return handshakeResponse.Err
}
}
authSendReceiver := func(authBytes []byte) (*SaslAuthenticateResponse, error) {
authenticateRequest := b.createSaslAuthenticateRequest(authBytes)
authenticateResponse := new(SaslAuthenticateResponse)
prom := makeResponsePromise(authenticateResponse.version())
authErr := b.sendInternal(authenticateRequest, prom)
if authErr != nil {
Logger.Printf("Error while performing SASL Auth %s\n", b.addr)
return nil, authErr
}
authErr = handleResponsePromise(authenticateRequest, authenticateResponse, prom, metricRegistry)
if authErr != nil {
Logger.Printf("Error while performing SASL Auth %s: %s\n", b.addr, authErr)
return nil, authErr
}
if !errors.Is(authenticateResponse.Err, ErrNoError) {
var err error = authenticateResponse.Err
if authenticateResponse.ErrorMessage != nil {
err = Wrap(authenticateResponse.Err, errors.New(*authenticateResponse.ErrorMessage))
}
return nil, err
}
b.computeSaslSessionLifetime(authenticateResponse)
return authenticateResponse, nil
}
switch b.conf.Net.SASL.Mechanism {
case SASLTypeOAuth:
provider := b.conf.Net.SASL.TokenProvider
return b.sendAndReceiveSASLOAuth(authSendReceiver, provider)
case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512:
return b.sendAndReceiveSASLSCRAMv1(authSendReceiver, b.conf.Net.SASL.SCRAMClientGeneratorFunc())
default:
return b.sendAndReceiveSASLPlainAuthV1(authSendReceiver)
}
}
func (b *Broker) sendAndReceiveKerberos() error {
b.kerberosAuthenticator.Config = &b.conf.Net.SASL.GSSAPI
if b.kerberosAuthenticator.NewKerberosClientFunc == nil {
b.kerberosAuthenticator.NewKerberosClientFunc = NewKerberosClient
}
return b.kerberosAuthenticator.Authorize(b)
}
func (b *Broker) sendAndReceiveSASLHandshake(saslType SASLMechanism, version int16) error {
rb := &SaslHandshakeRequest{Mechanism: string(saslType), Version: version}
req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
buf, err := encode(req, b.metricRegistry)
if err != nil {
return err
}
requestTime := time.Now()
// Will be decremented in updateIncomingCommunicationMetrics (except error)
b.addRequestInFlightMetrics(1)
bytes, err := b.write(buf)
b.updateOutgoingCommunicationMetrics(bytes)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to send SASL handshake %s: %s\n", b.addr, err.Error())
return err
}
b.correlationID++
header := make([]byte, 8) // response header
_, err = b.readFull(header)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to read SASL handshake header : %s\n", err.Error())
return err
}
length := binary.BigEndian.Uint32(header[:4])
payload := make([]byte, length-4)
n, err := b.readFull(payload)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to read SASL handshake payload : %s\n", err.Error())
return err
}
b.updateIncomingCommunicationMetrics(n+8, time.Since(requestTime))
res := &SaslHandshakeResponse{}
err = versionedDecode(payload, res, 0, b.metricRegistry)
if err != nil {
Logger.Printf("Failed to parse SASL handshake : %s\n", err.Error())
return err
}
if !errors.Is(res.Err, ErrNoError) {
Logger.Printf("Invalid SASL Mechanism : %s\n", res.Err.Error())
return res.Err
}
DebugLogger.Print("Completed pre-auth SASL handshake. Available mechanisms: ", res.EnabledMechanisms)
return nil
}
//
// In SASL Plain, Kafka expects the auth header to be in the following format
// Message format (from https://tools.ietf.org/html/rfc4616):
//
// message = [authzid] UTF8NUL authcid UTF8NUL passwd
// authcid = 1*SAFE ; MUST accept up to 255 octets
// authzid = 1*SAFE ; MUST accept up to 255 octets
// passwd = 1*SAFE ; MUST accept up to 255 octets
// UTF8NUL = %x00 ; UTF-8 encoded NUL character
//
// SAFE = UTF1 / UTF2 / UTF3 / UTF4
// ;; any UTF-8 encoded Unicode character except NUL
//
//
// Kafka 0.10.x supported SASL PLAIN/Kerberos via KAFKA-3149 (KIP-43).
// sendAndReceiveSASLPlainAuthV0 flows the v0 sasl auth NOT wrapped in the kafka protocol
//
// With SASL v0 handshake and auth then:
// When credentials are valid, Kafka returns a 4 byte array of null characters.
// When credentials are invalid, Kafka closes the connection.
func (b *Broker) sendAndReceiveSASLPlainAuthV0() error {
// default to V0 to allow for backward compatibility when SASL is enabled
// but not the handshake
if b.conf.Net.SASL.Handshake {
handshakeErr := b.sendAndReceiveSASLHandshake(SASLTypePlaintext, b.conf.Net.SASL.Version)
if handshakeErr != nil {
Logger.Printf("Error while performing SASL handshake %s: %s\n", b.addr, handshakeErr)
return handshakeErr
}
}
length := len(b.conf.Net.SASL.AuthIdentity) + 1 + len(b.conf.Net.SASL.User) + 1 + len(b.conf.Net.SASL.Password)
authBytes := make([]byte, length+4) // 4 byte length header + auth data
binary.BigEndian.PutUint32(authBytes, uint32(length))
copy(authBytes[4:], b.conf.Net.SASL.AuthIdentity+"\x00"+b.conf.Net.SASL.User+"\x00"+b.conf.Net.SASL.Password)
requestTime := time.Now()
// Will be decremented in updateIncomingCommunicationMetrics (except error)
b.addRequestInFlightMetrics(1)
bytesWritten, err := b.write(authBytes)
b.updateOutgoingCommunicationMetrics(bytesWritten)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error())
return err
}
header := make([]byte, 4)
n, err := b.readFull(header)
b.updateIncomingCommunicationMetrics(n, time.Since(requestTime))
// If the credentials are valid, we would get a 4 byte response filled with null characters.
// Otherwise, the broker closes the connection and we get an EOF
if err != nil {
Logger.Printf("Failed to read response while authenticating with SASL to broker %s: %s\n", b.addr, err.Error())
return err
}
DebugLogger.Printf("SASL authentication successful with broker %s:%v - %v\n", b.addr, n, header)
return nil
}
// Kafka 1.x.x onward added a SaslAuthenticate request/response message which
// wraps the SASL flow in the Kafka protocol, which allows for returning
// meaningful errors on authentication failure.
func (b *Broker) sendAndReceiveSASLPlainAuthV1(authSendReceiver func(authBytes []byte) (*SaslAuthenticateResponse, error)) error {
authBytes := []byte(b.conf.Net.SASL.AuthIdentity + "\x00" + b.conf.Net.SASL.User + "\x00" + b.conf.Net.SASL.Password)
_, err := authSendReceiver(authBytes)
return err
}
func currentUnixMilli() int64 {
return time.Now().UnixNano() / int64(time.Millisecond)
}
// sendAndReceiveSASLOAuth performs the authentication flow as described by KIP-255
// https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75968876
func (b *Broker) sendAndReceiveSASLOAuth(authSendReceiver func(authBytes []byte) (*SaslAuthenticateResponse, error), provider AccessTokenProvider) error {
token, err := provider.Token()
if err != nil {
return err
}
message, err := buildClientFirstMessage(token)
if err != nil {
return err
}
res, err := authSendReceiver(message)
if err != nil {
return err
}
isChallenge := len(res.SaslAuthBytes) > 0
if isChallenge {
// Abort the token exchange. The broker returns the failure code.
_, err = authSendReceiver([]byte(`\x01`))
}
return err
}
func (b *Broker) sendAndReceiveSASLSCRAMv0() error {
if err := b.sendAndReceiveSASLHandshake(b.conf.Net.SASL.Mechanism, SASLHandshakeV0); err != nil {
return err
}
scramClient := b.conf.Net.SASL.SCRAMClientGeneratorFunc()
if err := scramClient.Begin(b.conf.Net.SASL.User, b.conf.Net.SASL.Password, b.conf.Net.SASL.SCRAMAuthzID); err != nil {
return fmt.Errorf("failed to start SCRAM exchange with the server: %w", err)
}
msg, err := scramClient.Step("")
if err != nil {
return fmt.Errorf("failed to advance the SCRAM exchange: %w", err)
}
for !scramClient.Done() {
requestTime := time.Now()
// Will be decremented in updateIncomingCommunicationMetrics (except error)
b.addRequestInFlightMetrics(1)
length := len(msg)
authBytes := make([]byte, length+4) // 4 byte length header + auth data
binary.BigEndian.PutUint32(authBytes, uint32(length))
copy(authBytes[4:], msg)
_, err := b.write(authBytes)
b.updateOutgoingCommunicationMetrics(length + 4)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error())
return err
}
b.correlationID++
header := make([]byte, 4)
_, err = b.readFull(header)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to read response header while authenticating with SASL to broker %s: %s\n", b.addr, err.Error())
return err
}
payload := make([]byte, int32(binary.BigEndian.Uint32(header)))
n, err := b.readFull(payload)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to read response payload while authenticating with SASL to broker %s: %s\n", b.addr, err.Error())
return err
}
b.updateIncomingCommunicationMetrics(n+4, time.Since(requestTime))
msg, err = scramClient.Step(string(payload))
if err != nil {
Logger.Println("SASL authentication failed", err)
return err
}
}
DebugLogger.Println("SASL authentication succeeded")
return nil
}
func (b *Broker) sendAndReceiveSASLSCRAMv1(authSendReceiver func(authBytes []byte) (*SaslAuthenticateResponse, error), scramClient SCRAMClient) error {
if err := scramClient.Begin(b.conf.Net.SASL.User, b.conf.Net.SASL.Password, b.conf.Net.SASL.SCRAMAuthzID); err != nil {
return fmt.Errorf("failed to start SCRAM exchange with the server: %w", err)
}
msg, err := scramClient.Step("")
if err != nil {
return fmt.Errorf("failed to advance the SCRAM exchange: %w", err)
}
for !scramClient.Done() {
res, err := authSendReceiver([]byte(msg))
if err != nil {
return err
}
msg, err = scramClient.Step(string(res.SaslAuthBytes))
if err != nil {
Logger.Println("SASL authentication failed", err)
return err
}
}
DebugLogger.Println("SASL authentication succeeded")
return nil
}
func (b *Broker) createSaslAuthenticateRequest(msg []byte) *SaslAuthenticateRequest {
authenticateRequest := SaslAuthenticateRequest{SaslAuthBytes: msg}
if b.conf.Version.IsAtLeast(V2_2_0_0) {
authenticateRequest.Version = 1
}
return &authenticateRequest
}
// Build SASL/OAUTHBEARER initial client response as described by RFC-7628
// https://tools.ietf.org/html/rfc7628
func buildClientFirstMessage(token *AccessToken) ([]byte, error) {
var ext string
if token == nil {
return []byte{}, fmt.Errorf("failed to build client first message: token is nil")
}
if len(token.Extensions) > 0 {
if _, ok := token.Extensions[SASLExtKeyAuth]; ok {
return []byte{}, fmt.Errorf("the extension `%s` is invalid", SASLExtKeyAuth)
}
ext = "\x01" + mapToString(token.Extensions, "=", "\x01")
}
resp := []byte(fmt.Sprintf("n,,\x01auth=Bearer %s%s\x01\x01", token.Token, ext))
return resp, nil
}
// mapToString returns a list of key-value pairs ordered by key.
// keyValSep separates the key from the value. elemSep separates each pair.
func mapToString(extensions map[string]string, keyValSep string, elemSep string) string {
buf := make([]string, 0, len(extensions))
for k, v := range extensions {
buf = append(buf, k+keyValSep+v)
}
sort.Strings(buf)
return strings.Join(buf, elemSep)
}
func (b *Broker) computeSaslSessionLifetime(res *SaslAuthenticateResponse) {
if res.SessionLifetimeMs > 0 {
// Follows the Java Kafka implementation from SaslClientAuthenticator.ReauthInfo#setAuthenticationEndAndSessionReauthenticationTimes
// pick a random percentage between 85% and 95% for session re-authentication
positiveSessionLifetimeMs := res.SessionLifetimeMs
authenticationEndMs := currentUnixMilli()
pctWindowFactorToTakeNetworkLatencyAndClockDriftIntoAccount := 0.85
pctWindowJitterToAvoidReauthenticationStormAcrossManyChannelsSimultaneously := 0.10
pctToUse := pctWindowFactorToTakeNetworkLatencyAndClockDriftIntoAccount + rand.Float64()*pctWindowJitterToAvoidReauthenticationStormAcrossManyChannelsSimultaneously
sessionLifetimeMsToUse := int64(float64(positiveSessionLifetimeMs) * pctToUse)
DebugLogger.Printf("Session expiration in %d ms and session re-authentication on or after %d ms", positiveSessionLifetimeMs, sessionLifetimeMsToUse)
b.clientSessionReauthenticationTimeMs = authenticationEndMs + sessionLifetimeMsToUse
} else {
b.clientSessionReauthenticationTimeMs = 0
}
}
func (b *Broker) updateIncomingCommunicationMetrics(bytes int, requestLatency time.Duration) {
b.updateRequestLatencyAndInFlightMetrics(requestLatency)
b.responseRate.Mark(1)
if b.brokerResponseRate != nil {
b.brokerResponseRate.Mark(1)
}
responseSize := int64(bytes)
b.incomingByteRate.Mark(responseSize)
if b.brokerIncomingByteRate != nil {
b.brokerIncomingByteRate.Mark(responseSize)
}
b.responseSize.Update(responseSize)
if b.brokerResponseSize != nil {
b.brokerResponseSize.Update(responseSize)
}
}
func (b *Broker) updateRequestLatencyAndInFlightMetrics(requestLatency time.Duration) {
requestLatencyInMs := int64(requestLatency / time.Millisecond)
b.requestLatency.Update(requestLatencyInMs)
if b.brokerRequestLatency != nil {
b.brokerRequestLatency.Update(requestLatencyInMs)
}
b.addRequestInFlightMetrics(-1)
}
func (b *Broker) addRequestInFlightMetrics(i int64) {
b.requestsInFlight.Inc(i)
if b.brokerRequestsInFlight != nil {
b.brokerRequestsInFlight.Inc(i)
}
}
func (b *Broker) updateOutgoingCommunicationMetrics(bytes int) {
b.requestRate.Mark(1)
if b.brokerRequestRate != nil {
b.brokerRequestRate.Mark(1)
}
requestSize := int64(bytes)
b.outgoingByteRate.Mark(requestSize)
if b.brokerOutgoingByteRate != nil {
b.brokerOutgoingByteRate.Mark(requestSize)
}
b.requestSize.Update(requestSize)
if b.brokerRequestSize != nil {
b.brokerRequestSize.Update(requestSize)
}
}
func (b *Broker) updateProtocolMetrics(rb protocolBody) {
protocolRequestsRate := b.protocolRequestsRate[rb.key()]
if protocolRequestsRate == nil {
protocolRequestsRate = metrics.GetOrRegisterMeter(fmt.Sprintf("protocol-requests-rate-%d", rb.key()), b.metricRegistry)
b.protocolRequestsRate[rb.key()] = protocolRequestsRate
}
protocolRequestsRate.Mark(1)
if b.brokerProtocolRequestsRate != nil {
brokerProtocolRequestsRate := b.brokerProtocolRequestsRate[rb.key()]
if brokerProtocolRequestsRate == nil {
brokerProtocolRequestsRate = b.registerMeter(fmt.Sprintf("protocol-requests-rate-%d", rb.key()))
b.brokerProtocolRequestsRate[rb.key()] = brokerProtocolRequestsRate
}
brokerProtocolRequestsRate.Mark(1)
}
}
type throttleSupport interface {
throttleTime() time.Duration
}
func (b *Broker) handleThrottledResponse(resp protocolBody) {
throttledResponse, ok := resp.(throttleSupport)
if !ok {
return
}
throttleTime := throttledResponse.throttleTime()
if throttleTime == time.Duration(0) {
return
}
DebugLogger.Printf(
"broker/%d %T throttled %v\n", b.ID(), resp, throttleTime)
b.setThrottle(throttleTime)
b.updateThrottleMetric(throttleTime)
}
func (b *Broker) setThrottle(throttleTime time.Duration) {
b.throttleTimerLock.Lock()
defer b.throttleTimerLock.Unlock()
if b.throttleTimer != nil {
// if there is an existing timer stop/clear it
if !b.throttleTimer.Stop() {
<-b.throttleTimer.C
}
}
b.throttleTimer = time.NewTimer(throttleTime)
}
func (b *Broker) waitIfThrottled() {
b.throttleTimerLock.Lock()
defer b.throttleTimerLock.Unlock()
if b.throttleTimer != nil {
DebugLogger.Printf("broker/%d waiting for throttle timer\n", b.ID())
<-b.throttleTimer.C
b.throttleTimer = nil
}
}
func (b *Broker) updateThrottleMetric(throttleTime time.Duration) {
if b.brokerThrottleTime != nil {
throttleTimeInMs := int64(throttleTime / time.Millisecond)
b.brokerThrottleTime.Update(throttleTimeInMs)
}
}
func (b *Broker) registerMetrics() {
b.brokerIncomingByteRate = b.registerMeter("incoming-byte-rate")
b.brokerRequestRate = b.registerMeter("request-rate")
b.brokerFetchRate = b.registerMeter("consumer-fetch-rate")
b.brokerRequestSize = b.registerHistogram("request-size")
b.brokerRequestLatency = b.registerHistogram("request-latency-in-ms")
b.brokerOutgoingByteRate = b.registerMeter("outgoing-byte-rate")
b.brokerResponseRate = b.registerMeter("response-rate")
b.brokerResponseSize = b.registerHistogram("response-size")
b.brokerRequestsInFlight = b.registerCounter("requests-in-flight")
b.brokerThrottleTime = b.registerHistogram("throttle-time-in-ms")
b.brokerProtocolRequestsRate = map[int16]metrics.Meter{}
}
func (b *Broker) registerMeter(name string) metrics.Meter {
nameForBroker := getMetricNameForBroker(name, b)
return metrics.GetOrRegisterMeter(nameForBroker, b.metricRegistry)
}
func (b *Broker) registerHistogram(name string) metrics.Histogram {
nameForBroker := getMetricNameForBroker(name, b)
return getOrRegisterHistogram(nameForBroker, b.metricRegistry)
}
func (b *Broker) registerCounter(name string) metrics.Counter {
nameForBroker := getMetricNameForBroker(name, b)
return metrics.GetOrRegisterCounter(nameForBroker, b.metricRegistry)
}
func validServerNameTLS(addr string, cfg *tls.Config) *tls.Config {
if cfg == nil {
cfg = &tls.Config{
MinVersion: tls.VersionTLS12,
}
}
if cfg.ServerName != "" {
return cfg
}
c := cfg.Clone()
sn, _, err := net.SplitHostPort(addr)
if err != nil {
Logger.Println(fmt.Errorf("failed to get ServerName from addr %w", err))
}
c.ServerName = sn
return c
}
|