github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/error.go (about) 1 /** 2 * Copyright 2016 Confluent Inc. 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package kafka 18 19 // Automatically generate error codes from librdkafka 20 // See README for instructions 21 //go:generate $GOPATH/bin/go_rdkafka_generr generated_errors.go 22 23 /* 24 #include <stdlib.h> 25 #include "select_rdkafka.h" 26 */ 27 import "C" 28 29 import ( 30 "fmt" 31 "unsafe" 32 ) 33 34 // Error provides a Kafka-specific error container 35 type Error struct { 36 code ErrorCode 37 str string 38 fatal bool 39 retriable bool 40 txnRequiresAbort bool 41 } 42 43 func newError(code C.rd_kafka_resp_err_t) (err Error) { 44 return Error{code: ErrorCode(code)} 45 } 46 47 // NewError creates a new Error. 48 func NewError(code ErrorCode, str string, fatal bool) (err Error) { 49 return Error{code: code, str: str, fatal: fatal} 50 } 51 52 func newErrorFromString(code ErrorCode, str string) (err Error) { 53 return Error{code: code, str: str} 54 } 55 56 func newErrorFromCString(code C.rd_kafka_resp_err_t, cstr *C.char) (err Error) { 57 var str string 58 if cstr != nil { 59 str = C.GoString(cstr) 60 } else { 61 str = "" 62 } 63 return Error{code: ErrorCode(code), str: str} 64 } 65 66 func newCErrorFromString(code C.rd_kafka_resp_err_t, str string) (err Error) { 67 return newErrorFromString(ErrorCode(code), str) 68 } 69 70 // newErrorFromCError creates a new Error instance 71 func newErrorFromCError(cError *C.rd_kafka_error_t) Error { 72 return Error{ 73 code: ErrorCode(C.rd_kafka_error_code(cError)), 74 str: C.GoString(C.rd_kafka_error_string(cError)), 75 fatal: cint2bool(C.rd_kafka_error_is_fatal(cError)), 76 retriable: cint2bool(C.rd_kafka_error_is_retriable(cError)), 77 txnRequiresAbort: cint2bool(C.rd_kafka_error_txn_requires_abort(cError)), 78 } 79 } 80 81 // newErrorFromCErrorDestroy creates a new Error instance and destroys 82 // the passed cError. 83 func newErrorFromCErrorDestroy(cError *C.rd_kafka_error_t) Error { 84 defer C.rd_kafka_error_destroy(cError) 85 return newErrorFromCError(cError) 86 } 87 88 // Error returns a human readable representation of an Error 89 // Same as Error.String() 90 func (e Error) Error() string { 91 return e.String() 92 } 93 94 // String returns a human readable representation of an Error 95 func (e Error) String() string { 96 var errstr string 97 if len(e.str) > 0 { 98 errstr = e.str 99 } else { 100 errstr = e.code.String() 101 } 102 103 if e.IsFatal() { 104 return fmt.Sprintf("Fatal error: %s", errstr) 105 } 106 107 return errstr 108 } 109 110 // Code returns the ErrorCode of an Error 111 func (e Error) Code() ErrorCode { 112 return e.code 113 } 114 115 // IsFatal returns true if the error is a fatal error. 116 // A fatal error indicates the client instance is no longer operable and 117 // should be terminated. Typical causes include non-recoverable 118 // idempotent producer errors. 119 func (e Error) IsFatal() bool { 120 return e.fatal 121 } 122 123 // IsRetriable returns true if the operation that caused this error 124 // may be retried. 125 // This flag is currently only set by the Transactional producer API. 126 func (e Error) IsRetriable() bool { 127 return e.retriable 128 } 129 130 // TxnRequiresAbort returns true if the error is an abortable transaction error 131 // that requires the application to abort the current transaction with 132 // AbortTransaction() and start a new transaction with BeginTransaction() 133 // if it wishes to proceed with transactional operations. 134 // This flag is only set by the Transactional producer API. 135 func (e Error) TxnRequiresAbort() bool { 136 return e.txnRequiresAbort 137 } 138 139 // getFatalError returns an Error object if the client instance has raised a fatal error, else nil. 140 func getFatalError(H Handle) error { 141 cErrstr := (*C.char)(C.malloc(C.size_t(512))) 142 defer C.free(unsafe.Pointer(cErrstr)) 143 144 cErr := C.rd_kafka_fatal_error(H.gethandle().rk, cErrstr, 512) 145 if int(cErr) == 0 { 146 return nil 147 } 148 149 err := newErrorFromCString(cErr, cErrstr) 150 err.fatal = true 151 152 return err 153 } 154 155 // testFatalError triggers a fatal error in the underlying client. 156 // This is to be used strictly for testing purposes. 157 func testFatalError(H Handle, code ErrorCode, str string) ErrorCode { 158 return ErrorCode(C.rd_kafka_test_fatal_error(H.gethandle().rk, C.rd_kafka_resp_err_t(code), C.CString(str))) 159 }