C++98 and C99 fixes
This commit is contained in:
Родитель
2e3d6a8559
Коммит
93d838cbc6
|
@ -4,5 +4,7 @@
|
|||
|
||||
set -e
|
||||
./configure --clean
|
||||
./configure --enable-werror --disable-optimization \
|
||||
--enable-sharedptr-debug #--enable-refcnt-debug
|
||||
#export CFLAGS='-std=c99 -pedantic -Wshadow'
|
||||
#export CXXFLAGS='-std=c++98 -pedantic'
|
||||
./configure --enable-werror --disable-optimization
|
||||
#--enable-sharedptr-debug #--enable-refcnt-debug
|
||||
|
|
|
@ -326,8 +326,6 @@ int main (int argc, char **argv) {
|
|||
int pass;
|
||||
|
||||
for (pass = 0 ; pass < 2 ; pass++) {
|
||||
int i;
|
||||
|
||||
if (pass == 0) {
|
||||
arr = rd_kafka_conf_dump(conf, &cnt);
|
||||
printf("# Global config\n");
|
||||
|
|
|
@ -62,8 +62,8 @@ static bool exit_eof = false;
|
|||
static int eof_cnt = 0;
|
||||
static int partition_cnt = 0;
|
||||
static int verbosity = 1;
|
||||
static long long msg_cnt = 0;
|
||||
static long long msg_bytes = 0;
|
||||
static long msg_cnt = 0;
|
||||
static int64_t msg_bytes = 0;
|
||||
static void sigterm (int sig) {
|
||||
run = false;
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@
|
|||
|
||||
RdKafka::Producer::~Producer () {
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
static void dr_msg_cb_trampoline (rd_kafka_t *rk,
|
||||
const rd_kafka_message_t *
|
||||
|
|
|
@ -767,5 +767,5 @@ class RD_EXPORT Producer : public virtual Handle {
|
|||
};
|
||||
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -36,7 +36,7 @@
|
|||
|
||||
extern "C" {
|
||||
#include "../src/rdkafka.h"
|
||||
};
|
||||
}
|
||||
|
||||
#ifdef _MSC_VER
|
||||
typedef int mode_t;
|
||||
|
@ -483,4 +483,4 @@ class ProducerImpl : virtual public Producer, virtual public HandleImpl {
|
|||
|
||||
|
||||
|
||||
};
|
||||
}
|
||||
|
|
|
@ -103,7 +103,7 @@ const char *rd_addrinfo_prepare (const char *nodesvc,
|
|||
typedef struct rd_sockaddr_list_s {
|
||||
int rsal_cnt;
|
||||
int rsal_curr;
|
||||
rd_sockaddr_inx_t rsal_addr[0];
|
||||
rd_sockaddr_inx_t rsal_addr[];
|
||||
} rd_sockaddr_list_t;
|
||||
|
||||
|
||||
|
|
|
@ -49,7 +49,7 @@ void *rd_gz_decompress (void *compressed, int compressed_len,
|
|||
pass++;
|
||||
|
||||
for (; pass <= 2 ; pass++) {
|
||||
z_stream strm = {};
|
||||
z_stream strm = RD_ZERO_INIT;
|
||||
gz_header hdr;
|
||||
char buf[512];
|
||||
char *p;
|
||||
|
|
|
@ -879,7 +879,7 @@ static void rd_kafka_term_sig_handler (int sig) {
|
|||
}
|
||||
|
||||
static void rd_kafka_global_init (void) {
|
||||
static const char null_bytes_data[4] = { 0xff, 0xff, 0xff, 0xff };
|
||||
static const unsigned char null_bytes_data[4]={0xff, 0xff, 0xff, 0xff};
|
||||
|
||||
#if ENABLE_SHAREDPTR_DEBUG
|
||||
LIST_INIT(&rd_shared_ptr_debug_list);
|
||||
|
@ -1894,7 +1894,6 @@ static void rd_kafka_dump0 (FILE *fp, rd_kafka_t *rk, int locks) {
|
|||
rd_kafka_toppar_dump(fp, " ",
|
||||
rd_kafka_toppar_s2i(rkt->rkt_ua));
|
||||
if (rd_list_empty(&rkt->rkt_desp)) {
|
||||
int i;
|
||||
fprintf(fp, " desired partitions:");
|
||||
RD_LIST_FOREACH(s_rktp, &rkt->rkt_desp, i)
|
||||
fprintf(fp, " %"PRId32,
|
||||
|
|
|
@ -436,5 +436,5 @@ void rd_kafka_assignors_init (rd_kafka_t *rk) {
|
|||
*/
|
||||
void rd_kafka_assignors_term (rd_kafka_t *rk) {
|
||||
rd_list_destroy(&rk->rk_conf.partition_assignors,
|
||||
(void *)rd_kafka_assignor_destroy);
|
||||
(void *)(void *)rd_kafka_assignor_destroy);
|
||||
}
|
||||
|
|
|
@ -1506,7 +1506,7 @@ static int rd_kafka_compress_MessageSet_buf (rd_kafka_broker_t *rkb,
|
|||
MessageSetSize,
|
||||
RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
|
||||
rktp->rktp_partition,
|
||||
strm.msg ? : "", r);
|
||||
strm.msg ? strm.msg : "", r);
|
||||
return -1;
|
||||
}
|
||||
|
||||
|
@ -1544,7 +1544,7 @@ static int rd_kafka_compress_MessageSet_buf (rd_kafka_broker_t *rkb,
|
|||
rktp_rkt->
|
||||
rkt_topic),
|
||||
rktp->rktp_partition,
|
||||
strm.msg ? : "", r);
|
||||
strm.msg ? strm.msg : "", r);
|
||||
deflateEnd(&strm);
|
||||
rd_free(siov.iov_base);
|
||||
return -1;
|
||||
|
@ -1565,7 +1565,7 @@ static int rd_kafka_compress_MessageSet_buf (rd_kafka_broker_t *rkb,
|
|||
RD_KAFKAP_STR_PR(rktp->rktp_rkt->
|
||||
rkt_topic),
|
||||
rktp->rktp_partition,
|
||||
strm.msg ? : "", r);
|
||||
strm.msg ? strm.msg : "", r);
|
||||
deflateEnd(&strm);
|
||||
rd_free(siov.iov_base);
|
||||
return -1;
|
||||
|
@ -2388,7 +2388,7 @@ static rd_kafka_resp_err_t rd_kafka_messageset_handle (rd_kafka_broker_t *rkb,
|
|||
{
|
||||
const char *inbuf = Value.data;
|
||||
int r;
|
||||
static const char snappy_java_magic[] =
|
||||
static const unsigned char snappy_java_magic[] =
|
||||
{ 0x82, 'S','N','A','P','P','Y', 0 };
|
||||
static const int snappy_java_hdrlen = 8+4+4;
|
||||
|
||||
|
|
|
@ -33,7 +33,7 @@ extern const char *rd_kafka_secproto_names[];
|
|||
|
||||
#define RD_KAFKA_NODENAME_SIZE 128
|
||||
|
||||
typedef struct rd_kafka_broker_s {
|
||||
struct rd_kafka_broker_s { /* rd_kafka_broker_t */
|
||||
TAILQ_ENTRY(rd_kafka_broker_s) rkb_link;
|
||||
|
||||
int32_t rkb_nodeid;
|
||||
|
@ -148,7 +148,7 @@ typedef struct rd_kafka_broker_s {
|
|||
char msg[512];
|
||||
int err; /* errno */
|
||||
} rkb_err;
|
||||
} rd_kafka_broker_t;
|
||||
};
|
||||
|
||||
#define rd_kafka_broker_keep(rkb) rd_refcnt_add(&(rkb)->rkb_refcnt)
|
||||
#define rd_kafka_broker_lock(rkb) mtx_lock(&(rkb)->rkb_lock)
|
||||
|
|
|
@ -201,7 +201,7 @@ typedef void (rd_kafka_resp_cb_t) (rd_kafka_broker_t *rkb,
|
|||
rd_kafka_buf_t *request,
|
||||
void *opaque);
|
||||
|
||||
typedef struct rd_kafka_buf_s {
|
||||
struct rd_kafka_buf_s { /* rd_kafka_buf_t */
|
||||
TAILQ_ENTRY(rd_kafka_buf_s) rkbuf_link;
|
||||
|
||||
int32_t rkbuf_corrid;
|
||||
|
@ -262,7 +262,7 @@ typedef struct rd_kafka_buf_s {
|
|||
int64_t rkbuf_offset; /* Used by OffsetCommit */
|
||||
|
||||
rd_kafka_msgq_t rkbuf_msgq;
|
||||
} rd_kafka_buf_t;
|
||||
};
|
||||
|
||||
|
||||
typedef struct rd_kafka_bufq_s {
|
||||
|
|
|
@ -75,7 +75,7 @@ const char *rd_kafka_offset2str (int64_t offset) {
|
|||
i = (i + 1) % 16;
|
||||
|
||||
if (offset >= 0)
|
||||
snprintf(ret[i], sizeof(ret[i]), "%"PRId64, offset);
|
||||
rd_snprintf(ret[i], sizeof(ret[i]), "%"PRId64, offset);
|
||||
else if (offset == RD_KAFKA_OFFSET_BEGINNING)
|
||||
return "BEGINNING";
|
||||
else if (offset == RD_KAFKA_OFFSET_END)
|
||||
|
@ -85,10 +85,10 @@ const char *rd_kafka_offset2str (int64_t offset) {
|
|||
else if (offset == RD_KAFKA_OFFSET_ERROR)
|
||||
return "ERROR";
|
||||
else if (offset <= RD_KAFKA_OFFSET_TAIL_BASE)
|
||||
snprintf(ret[i], sizeof(ret[i]), "TAIL(%lld)",
|
||||
llabs(offset - RD_KAFKA_OFFSET_TAIL_BASE));
|
||||
rd_snprintf(ret[i], sizeof(ret[i]), "TAIL(%lld)",
|
||||
llabs(offset - RD_KAFKA_OFFSET_TAIL_BASE));
|
||||
else
|
||||
snprintf(ret[i], sizeof(ret[i]), "%"PRId64"?", offset);
|
||||
rd_snprintf(ret[i], sizeof(ret[i]), "%"PRId64"?", offset);
|
||||
|
||||
return ret[i];
|
||||
}
|
||||
|
|
|
@ -1542,8 +1542,7 @@ rd_kafka_topic_partition_list_add0 (rd_kafka_topic_partition_list_t *rktparlist,
|
|||
void
|
||||
rd_kafka_topic_partition_list_add (rd_kafka_topic_partition_list_t *rktparlist,
|
||||
const char *topic, int32_t partition) {
|
||||
return rd_kafka_topic_partition_list_add0(rktparlist, topic, partition,
|
||||
NULL);
|
||||
rd_kafka_topic_partition_list_add0(rktparlist, topic, partition, NULL);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ extern const char *rd_kafka_fetch_states[];
|
|||
/**
|
||||
* Topic + Partition combination
|
||||
*/
|
||||
typedef struct rd_kafka_toppar_s {
|
||||
struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */
|
||||
TAILQ_ENTRY(rd_kafka_toppar_s) rktp_rklink; /* rd_kafka_t link */
|
||||
TAILQ_ENTRY(rd_kafka_toppar_s) rktp_rkblink; /* rd_kafka_broker_t link*/
|
||||
CIRCLEQ_ENTRY(rd_kafka_toppar_s) rktp_fetchlink; /* rkb_fetch_toppars */
|
||||
|
@ -142,7 +142,7 @@ typedef struct rd_kafka_toppar_s {
|
|||
rd_atomic64_t rx_ver_drops;
|
||||
} rktp_c;
|
||||
|
||||
} rd_kafka_toppar_t;
|
||||
};
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -263,7 +263,7 @@ typedef struct rd_kafkap_bytes_s {
|
|||
int len; /* Kafka bytes length (-1=NULL, 0=empty, >0=data) */
|
||||
const void *data; /* points just past the struct, or other memory,
|
||||
* not NULL-terminated */
|
||||
const char _data[0]; /* Bytes following struct when new()ed */
|
||||
const char _data[]; /* Bytes following struct when new()ed */
|
||||
} rd_kafkap_bytes_t;
|
||||
|
||||
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
|
||||
#include "rdkafka_op.h"
|
||||
|
||||
typedef struct rd_kafka_q_s {
|
||||
struct rd_kafka_q_s {
|
||||
mtx_t rkq_lock;
|
||||
cnd_t rkq_cond;
|
||||
struct rd_kafka_q_s *rkq_fwdq; /* Forwarded/Routed queue.
|
||||
|
@ -18,7 +18,7 @@ typedef struct rd_kafka_q_s {
|
|||
* Flag is cleared on destory */
|
||||
|
||||
rd_kafka_t *rkq_rk;
|
||||
} rd_kafka_q_t;
|
||||
};
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -125,7 +125,7 @@ void test_conf_init (rd_kafka_conf_t **conf, rd_kafka_topic_conf_t **topic_conf,
|
|||
|
||||
#ifdef SIGIO
|
||||
/* Quick termination */
|
||||
snprintf(buf, sizeof(buf), "%i", SIGIO);
|
||||
rd_snprintf(buf, sizeof(buf), "%i", SIGIO);
|
||||
rd_kafka_conf_set(*conf, "internal.termination.signal",
|
||||
buf, NULL, 0);
|
||||
signal(SIGIO, SIG_IGN);
|
||||
|
|
Загрузка…
Ссылка в новой задаче