| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | // Copyright (c) 2015-2023 MinIO, Inc.
 | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | //
 | 
					
						
							|  |  |  | // This file is part of MinIO Object Storage stack
 | 
					
						
							|  |  |  | //
 | 
					
						
							|  |  |  | // 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 <http://www.gnu.org/licenses/>.
 | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | package kafka | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | import ( | 
					
						
							|  |  |  | 	"context" | 
					
						
							|  |  |  | 	"crypto/tls" | 
					
						
							|  |  |  | 	"crypto/x509" | 
					
						
							|  |  |  | 	"encoding/json" | 
					
						
							|  |  |  | 	"errors" | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	"fmt" | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	"net" | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	"os" | 
					
						
							|  |  |  | 	"path/filepath" | 
					
						
							| 
									
										
										
										
											2022-02-25 01:05:33 +08:00
										 |  |  | 	"sync" | 
					
						
							| 
									
										
										
										
											2022-11-11 02:20:21 +08:00
										 |  |  | 	"sync/atomic" | 
					
						
							| 
									
										
										
										
											2023-04-18 06:45:01 +08:00
										 |  |  | 	"time" | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-08-23 11:26:35 +08:00
										 |  |  | 	"github.com/IBM/sarama" | 
					
						
							|  |  |  | 	saramatls "github.com/IBM/sarama/tools/tls" | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2022-02-25 01:05:33 +08:00
										 |  |  | 	"github.com/minio/minio/internal/logger/target/types" | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	"github.com/minio/minio/internal/once" | 
					
						
							|  |  |  | 	"github.com/minio/minio/internal/store" | 
					
						
							| 
									
										
										
										
											2023-09-05 03:57:37 +08:00
										 |  |  | 	xnet "github.com/minio/pkg/v2/net" | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | ) | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | // the suffix for the configured queue dir where the logs will be persisted.
 | 
					
						
							|  |  |  | const kafkaLoggerExtension = ".kafka.log" | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-09-22 07:58:24 +08:00
										 |  |  | const ( | 
					
						
							|  |  |  | 	statusClosed = iota | 
					
						
							|  |  |  | 	statusOffline | 
					
						
							|  |  |  | 	statusOnline | 
					
						
							|  |  |  | ) | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | // Config - kafka target arguments.
 | 
					
						
							|  |  |  | type Config struct { | 
					
						
							|  |  |  | 	Enabled bool        `json:"enable"` | 
					
						
							|  |  |  | 	Brokers []xnet.Host `json:"brokers"` | 
					
						
							|  |  |  | 	Topic   string      `json:"topic"` | 
					
						
							|  |  |  | 	Version string      `json:"version"` | 
					
						
							|  |  |  | 	TLS     struct { | 
					
						
							|  |  |  | 		Enable        bool               `json:"enable"` | 
					
						
							|  |  |  | 		RootCAs       *x509.CertPool     `json:"-"` | 
					
						
							|  |  |  | 		SkipVerify    bool               `json:"skipVerify"` | 
					
						
							|  |  |  | 		ClientAuth    tls.ClientAuthType `json:"clientAuth"` | 
					
						
							|  |  |  | 		ClientTLSCert string             `json:"clientTLSCert"` | 
					
						
							|  |  |  | 		ClientTLSKey  string             `json:"clientTLSKey"` | 
					
						
							|  |  |  | 	} `json:"tls"` | 
					
						
							|  |  |  | 	SASL struct { | 
					
						
							|  |  |  | 		Enable    bool   `json:"enable"` | 
					
						
							|  |  |  | 		User      string `json:"username"` | 
					
						
							|  |  |  | 		Password  string `json:"password"` | 
					
						
							|  |  |  | 		Mechanism string `json:"mechanism"` | 
					
						
							|  |  |  | 	} `json:"sasl"` | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	// Queue store
 | 
					
						
							|  |  |  | 	QueueSize int    `json:"queueSize"` | 
					
						
							|  |  |  | 	QueueDir  string `json:"queueDir"` | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	// Custom logger
 | 
					
						
							| 
									
										
										
										
											2022-07-28 00:44:59 +08:00
										 |  |  | 	LogOnce func(ctx context.Context, err error, id string, errKind ...interface{}) `json:"-"` | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // Check if atleast one broker in cluster is active
 | 
					
						
							| 
									
										
										
										
											2023-04-18 06:45:01 +08:00
										 |  |  | func (k Config) pingBrokers() (err error) { | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	d := net.Dialer{Timeout: 1 * time.Second} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	errs := make([]error, len(k.Brokers)) | 
					
						
							|  |  |  | 	var wg sync.WaitGroup | 
					
						
							|  |  |  | 	for idx, broker := range k.Brokers { | 
					
						
							|  |  |  | 		broker := broker | 
					
						
							|  |  |  | 		idx := idx | 
					
						
							|  |  |  | 		wg.Add(1) | 
					
						
							|  |  |  | 		go func(broker xnet.Host, idx int) { | 
					
						
							|  |  |  | 			defer wg.Done() | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 			_, errs[idx] = d.Dial("tcp", broker.String()) | 
					
						
							|  |  |  | 		}(broker, idx) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	wg.Wait() | 
					
						
							| 
									
										
										
										
											2023-04-18 06:45:01 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	var retErr error | 
					
						
							|  |  |  | 	for _, err := range errs { | 
					
						
							|  |  |  | 		if err == nil { | 
					
						
							|  |  |  | 			// if one broker is online its enough
 | 
					
						
							|  |  |  | 			return nil | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 		retErr = err | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	return retErr | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | // Target - Kafka target.
 | 
					
						
							|  |  |  | type Target struct { | 
					
						
							| 
									
										
										
										
											2023-09-22 07:58:24 +08:00
										 |  |  | 	status int32 | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	totalMessages  int64 | 
					
						
							|  |  |  | 	failedMessages int64 | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	wg sync.WaitGroup | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	// Channel of log entries.
 | 
					
						
							|  |  |  | 	// Reading logCh must hold read lock on logChMu (to avoid read race)
 | 
					
						
							|  |  |  | 	// Sending a value on logCh must hold read lock on logChMu (to avoid closing)
 | 
					
						
							|  |  |  | 	logCh   chan interface{} | 
					
						
							|  |  |  | 	logChMu sync.RWMutex | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	// store to persist and replay the logs to the target
 | 
					
						
							|  |  |  | 	// to avoid missing events when the target is down.
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	store          store.Store[interface{}] | 
					
						
							|  |  |  | 	storeCtxCancel context.CancelFunc | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	initKafkaOnce      once.Init | 
					
						
							|  |  |  | 	initQueueStoreOnce once.Init | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	producer sarama.SyncProducer | 
					
						
							|  |  |  | 	kconfig  Config | 
					
						
							|  |  |  | 	config   *sarama.Config | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | func (h *Target) validate() error { | 
					
						
							|  |  |  | 	if len(h.kconfig.Brokers) == 0 { | 
					
						
							|  |  |  | 		return errors.New("no broker address found") | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	for _, b := range h.kconfig.Brokers { | 
					
						
							|  |  |  | 		if _, err := xnet.ParseHost(b.String()); err != nil { | 
					
						
							|  |  |  | 			return err | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2022-11-11 02:20:21 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	return nil | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // Name returns the name of the target
 | 
					
						
							|  |  |  | func (h *Target) Name() string { | 
					
						
							|  |  |  | 	return "minio-kafka-audit" | 
					
						
							| 
									
										
										
										
											2022-11-11 02:20:21 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | // Endpoint - return kafka target
 | 
					
						
							|  |  |  | func (h *Target) Endpoint() string { | 
					
						
							|  |  |  | 	return "kafka" | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // String - kafka string
 | 
					
						
							|  |  |  | func (h *Target) String() string { | 
					
						
							|  |  |  | 	return "kafka" | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | // Stats returns the target statistics.
 | 
					
						
							|  |  |  | func (h *Target) Stats() types.TargetStats { | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	h.logChMu.RLock() | 
					
						
							|  |  |  | 	queueLength := len(h.logCh) | 
					
						
							|  |  |  | 	h.logChMu.RUnlock() | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	return types.TargetStats{ | 
					
						
							|  |  |  | 		TotalMessages:  atomic.LoadInt64(&h.totalMessages), | 
					
						
							|  |  |  | 		FailedMessages: atomic.LoadInt64(&h.failedMessages), | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 		QueueLength:    queueLength, | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2022-11-29 00:03:26 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | // Init initialize kafka target
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | func (h *Target) Init(ctx context.Context) error { | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	if !h.kconfig.Enabled { | 
					
						
							|  |  |  | 		return nil | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	if err := h.validate(); err != nil { | 
					
						
							|  |  |  | 		return err | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	if h.kconfig.QueueDir != "" { | 
					
						
							|  |  |  | 		if err := h.initQueueStoreOnce.DoWithContext(ctx, h.initQueueStore); err != nil { | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 			return err | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 		return h.initKafkaOnce.Do(h.init) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	if err := h.init(); err != nil { | 
					
						
							|  |  |  | 		return err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	go h.startKakfaLogger() | 
					
						
							|  |  |  | 	return nil | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | func (h *Target) initQueueStore(ctx context.Context) (err error) { | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	var queueStore store.Store[interface{}] | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	queueDir := filepath.Join(h.kconfig.QueueDir, h.Name()) | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	queueStore = store.NewQueueStore[interface{}](queueDir, uint64(h.kconfig.QueueSize), kafkaLoggerExtension) | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	if err = queueStore.Open(); err != nil { | 
					
						
							|  |  |  | 		return fmt.Errorf("unable to initialize the queue store of %s webhook: %w", h.Name(), err) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	ctx, cancel := context.WithCancel(ctx) | 
					
						
							|  |  |  | 	h.store = queueStore | 
					
						
							|  |  |  | 	h.storeCtxCancel = cancel | 
					
						
							|  |  |  | 	store.StreamItems(h.store, h, ctx.Done(), h.kconfig.LogOnce) | 
					
						
							|  |  |  | 	return | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | func (h *Target) startKakfaLogger() { | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	h.logChMu.RLock() | 
					
						
							|  |  |  | 	logCh := h.logCh | 
					
						
							|  |  |  | 	if logCh != nil { | 
					
						
							|  |  |  | 		// We are not allowed to add when logCh is nil
 | 
					
						
							|  |  |  | 		h.wg.Add(1) | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 		defer h.wg.Done() | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 	h.logChMu.RUnlock() | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	if logCh == nil { | 
					
						
							|  |  |  | 		return | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// Create a routine which sends json logs received
 | 
					
						
							|  |  |  | 	// from an internal channel.
 | 
					
						
							|  |  |  | 	for entry := range logCh { | 
					
						
							|  |  |  | 		h.logEntry(entry) | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | func (h *Target) logEntry(entry interface{}) { | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	atomic.AddInt64(&h.totalMessages, 1) | 
					
						
							|  |  |  | 	if err := h.send(entry); err != nil { | 
					
						
							|  |  |  | 		atomic.AddInt64(&h.failedMessages, 1) | 
					
						
							|  |  |  | 		h.kconfig.LogOnce(context.Background(), err, h.kconfig.Topic) | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | func (h *Target) send(entry interface{}) error { | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	if err := h.initKafkaOnce.Do(h.init); err != nil { | 
					
						
							|  |  |  | 		return err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	logJSON, err := json.Marshal(&entry) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		return err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	msg := sarama.ProducerMessage{ | 
					
						
							|  |  |  | 		Topic: h.kconfig.Topic, | 
					
						
							|  |  |  | 		Value: sarama.ByteEncoder(logJSON), | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	_, _, err = h.producer.SendMessage(&msg) | 
					
						
							| 
									
										
										
										
											2023-09-22 07:58:24 +08:00
										 |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		atomic.StoreInt32(&h.status, statusOffline) | 
					
						
							|  |  |  | 	} else { | 
					
						
							|  |  |  | 		atomic.StoreInt32(&h.status, statusOnline) | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	return err | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // Init initialize kafka target
 | 
					
						
							|  |  |  | func (h *Target) init() error { | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	if err := h.kconfig.pingBrokers(); err != nil { | 
					
						
							|  |  |  | 		return err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	sconfig := sarama.NewConfig() | 
					
						
							|  |  |  | 	if h.kconfig.Version != "" { | 
					
						
							|  |  |  | 		kafkaVersion, err := sarama.ParseKafkaVersion(h.kconfig.Version) | 
					
						
							|  |  |  | 		if err != nil { | 
					
						
							|  |  |  | 			return err | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		sconfig.Version = kafkaVersion | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	sconfig.Net.KeepAlive = 60 * time.Second | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	sconfig.Net.SASL.User = h.kconfig.SASL.User | 
					
						
							|  |  |  | 	sconfig.Net.SASL.Password = h.kconfig.SASL.Password | 
					
						
							|  |  |  | 	initScramClient(h.kconfig, sconfig) // initializes configured scram client.
 | 
					
						
							|  |  |  | 	sconfig.Net.SASL.Enable = h.kconfig.SASL.Enable | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	tlsConfig, err := saramatls.NewConfig(h.kconfig.TLS.ClientTLSCert, h.kconfig.TLS.ClientTLSKey) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		return err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	sconfig.Net.TLS.Enable = h.kconfig.TLS.Enable | 
					
						
							|  |  |  | 	sconfig.Net.TLS.Config = tlsConfig | 
					
						
							|  |  |  | 	sconfig.Net.TLS.Config.InsecureSkipVerify = h.kconfig.TLS.SkipVerify | 
					
						
							|  |  |  | 	sconfig.Net.TLS.Config.ClientAuth = h.kconfig.TLS.ClientAuth | 
					
						
							|  |  |  | 	sconfig.Net.TLS.Config.RootCAs = h.kconfig.TLS.RootCAs | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-08-23 11:26:35 +08:00
										 |  |  | 	// These settings are needed to ensure that kafka client doesn't hang on brokers
 | 
					
						
							|  |  |  | 	// refer https://github.com/IBM/sarama/issues/765#issuecomment-254333355
 | 
					
						
							|  |  |  | 	sconfig.Producer.Retry.Max = 2 | 
					
						
							|  |  |  | 	sconfig.Producer.Retry.Backoff = (10 * time.Second) | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	sconfig.Producer.Return.Successes = true | 
					
						
							| 
									
										
										
										
											2023-08-23 11:26:35 +08:00
										 |  |  | 	sconfig.Producer.Return.Errors = true | 
					
						
							|  |  |  | 	sconfig.Producer.RequiredAcks = 1 | 
					
						
							|  |  |  | 	sconfig.Producer.Timeout = (10 * time.Second) | 
					
						
							|  |  |  | 	sconfig.Net.ReadTimeout = (10 * time.Second) | 
					
						
							|  |  |  | 	sconfig.Net.DialTimeout = (10 * time.Second) | 
					
						
							|  |  |  | 	sconfig.Net.WriteTimeout = (10 * time.Second) | 
					
						
							|  |  |  | 	sconfig.Metadata.Retry.Max = 1 | 
					
						
							|  |  |  | 	sconfig.Metadata.Retry.Backoff = (10 * time.Second) | 
					
						
							|  |  |  | 	sconfig.Metadata.RefreshFrequency = (15 * time.Minute) | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	h.config = sconfig | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	var brokers []string | 
					
						
							|  |  |  | 	for _, broker := range h.kconfig.Brokers { | 
					
						
							|  |  |  | 		brokers = append(brokers, broker.String()) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	producer, err := sarama.NewSyncProducer(brokers, sconfig) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		return err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	h.producer = producer | 
					
						
							| 
									
										
										
										
											2023-09-22 07:58:24 +08:00
										 |  |  | 	atomic.StoreInt32(&h.status, statusOnline) | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	return nil | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | // IsOnline returns true if the target is online.
 | 
					
						
							|  |  |  | func (h *Target) IsOnline(_ context.Context) bool { | 
					
						
							| 
									
										
										
										
											2023-09-22 07:58:24 +08:00
										 |  |  | 	return atomic.LoadInt32(&h.status) == statusOnline | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // Send log message 'e' to kafka target.
 | 
					
						
							|  |  |  | func (h *Target) Send(ctx context.Context, entry interface{}) error { | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	if h.store != nil { | 
					
						
							|  |  |  | 		// save the entry to the queue store which will be replayed to the target.
 | 
					
						
							|  |  |  | 		return h.store.Put(entry) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	h.logChMu.RLock() | 
					
						
							|  |  |  | 	defer h.logChMu.RUnlock() | 
					
						
							|  |  |  | 	if h.logCh == nil { | 
					
						
							|  |  |  | 		// We are closing...
 | 
					
						
							|  |  |  | 		return nil | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	select { | 
					
						
							|  |  |  | 	case h.logCh <- entry: | 
					
						
							| 
									
										
										
										
											2023-08-18 05:53:43 +08:00
										 |  |  | 	case <-ctx.Done(): | 
					
						
							|  |  |  | 		// return error only for context timedout.
 | 
					
						
							|  |  |  | 		if errors.Is(ctx.Err(), context.DeadlineExceeded) { | 
					
						
							|  |  |  | 			return ctx.Err() | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		return nil | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 	default: | 
					
						
							|  |  |  | 		// log channel is full, do not wait and return
 | 
					
						
							|  |  |  | 		// an error immediately to the caller
 | 
					
						
							|  |  |  | 		atomic.AddInt64(&h.totalMessages, 1) | 
					
						
							|  |  |  | 		atomic.AddInt64(&h.failedMessages, 1) | 
					
						
							|  |  |  | 		return errors.New("log buffer full") | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 	return nil | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // SendFromStore - reads the log from store and sends it to kafka.
 | 
					
						
							| 
									
										
										
										
											2023-10-07 23:07:38 +08:00
										 |  |  | func (h *Target) SendFromStore(key store.Key) (err error) { | 
					
						
							|  |  |  | 	auditEntry, err := h.store.Get(key.Name) | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		if os.IsNotExist(err) { | 
					
						
							|  |  |  | 			return nil | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		return err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	atomic.AddInt64(&h.totalMessages, 1) | 
					
						
							|  |  |  | 	err = h.send(auditEntry) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		atomic.AddInt64(&h.failedMessages, 1) | 
					
						
							|  |  |  | 		return | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	// Delete the event from store.
 | 
					
						
							| 
									
										
										
										
											2023-10-07 23:07:38 +08:00
										 |  |  | 	return h.store.Del(key.Name) | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2022-02-18 03:11:15 +08:00
										 |  |  | // Cancel - cancels the target
 | 
					
						
							|  |  |  | func (h *Target) Cancel() { | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	// If queuestore is configured, cancel it's context to
 | 
					
						
							|  |  |  | 	// stop the replay go-routine.
 | 
					
						
							|  |  |  | 	if h.store != nil { | 
					
						
							|  |  |  | 		h.storeCtxCancel() | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	// Set logch to nil and close it.
 | 
					
						
							|  |  |  | 	// This will block all Send operations,
 | 
					
						
							|  |  |  | 	// and finish the existing ones.
 | 
					
						
							|  |  |  | 	// All future ones will be discarded.
 | 
					
						
							|  |  |  | 	h.logChMu.Lock() | 
					
						
							|  |  |  | 	close(h.logCh) | 
					
						
							|  |  |  | 	h.logCh = nil | 
					
						
							|  |  |  | 	h.logChMu.Unlock() | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2023-05-09 12:20:31 +08:00
										 |  |  | 	if h.producer != nil { | 
					
						
							|  |  |  | 		h.producer.Close() | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	// Wait for messages to be sent...
 | 
					
						
							| 
									
										
										
										
											2022-02-25 01:05:33 +08:00
										 |  |  | 	h.wg.Wait() | 
					
						
							| 
									
										
										
										
											2022-02-18 03:11:15 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | // New initializes a new logger target which
 | 
					
						
							|  |  |  | // sends log over http to the specified endpoint
 | 
					
						
							|  |  |  | func New(config Config) *Target { | 
					
						
							|  |  |  | 	target := &Target{ | 
					
						
							| 
									
										
										
										
											2023-06-08 02:47:00 +08:00
										 |  |  | 		logCh:   make(chan interface{}, config.QueueSize), | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 		kconfig: config, | 
					
						
							| 
									
										
										
										
											2023-09-22 07:58:24 +08:00
										 |  |  | 		status:  statusOffline, | 
					
						
							| 
									
										
										
										
											2021-07-14 00:39:13 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 	return target | 
					
						
							|  |  |  | } | 
					
						
							| 
									
										
										
										
											2022-02-25 01:05:33 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | // Type - returns type of the target
 | 
					
						
							|  |  |  | func (h *Target) Type() types.TargetType { | 
					
						
							|  |  |  | 	return types.TargetKafka | 
					
						
							|  |  |  | } |