Codebase list kafkacat / 97fa05b
Fix indentation and whitespaces Magnus Edenhill 4 years ago
6 changed file(s) with 215 addition(s) and 187 deletion(s). Raw diff Collapse all Expand all
119119 */
120120 char *kc_avro_to_json (const void *data, size_t data_len,
121121 char *errstr, size_t errstr_size) {
122 avro_value_t avro;
123 serdes_schema_t *schema;
124 char *json;
125 serdes_err_t err;
122 avro_value_t avro;
123 serdes_schema_t *schema;
124 char *json;
125 serdes_err_t err;
126126
127 err = serdes_deserialize_avro(serdes, &avro, &schema, data, data_len,
128 errstr, errstr_size);
129 if (err) {
130 if (err == SERDES_ERR_FRAMING_INVALID ||
131 strstr(errstr, "Invalid CP1 magic byte")) {
132 static const char badframing[] =
133 ": message not produced with "
134 "Schema-Registry Avro framing";
135 int len = strlen(errstr);
127 err = serdes_deserialize_avro(serdes, &avro, &schema, data, data_len,
128 errstr, errstr_size);
129 if (err) {
130 if (err == SERDES_ERR_FRAMING_INVALID ||
131 strstr(errstr, "Invalid CP1 magic byte")) {
132 static const char badframing[] =
133 ": message not produced with "
134 "Schema-Registry Avro framing";
135 int len = strlen(errstr);
136136
137 if (len + sizeof(badframing) < errstr_size)
138 snprintf(errstr+len, errstr_size-len,
139 "%s", badframing);
140 }
141 return NULL;
142 }
137 if (len + sizeof(badframing) < errstr_size)
138 snprintf(errstr+len, errstr_size-len,
139 "%s", badframing);
140 }
141 return NULL;
142 }
143143
144 if (avro_value_to_json(&avro, 1/*one-line*/, &json)) {
145 snprintf(errstr, errstr_size, "Failed to encode Avro as JSON");
146 avro_value_decref(&avro);
147 return NULL;
148 }
144 if (avro_value_to_json(&avro, 1/*one-line*/, &json)) {
145 snprintf(errstr, errstr_size, "Failed to encode Avro as JSON");
146 avro_value_decref(&avro);
147 return NULL;
148 }
149149
150 avro_value_decref(&avro);
150 avro_value_decref(&avro);
151151
152 return json;
152 return json;
153153 }
3030 #include <yajl/yajl_gen.h>
3131
3232 #define JS_STR(G, STR) do { \
33 const char *_s = (STR); \
34 yajl_gen_string(G, (const unsigned char *)_s, strlen(_s)); \
33 const char *_s = (STR); \
34 yajl_gen_string(G, (const unsigned char *)_s, strlen(_s)); \
3535 } while (0)
3636
3737 void fmt_msg_output_json (FILE *fp, const rd_kafka_message_t *rkmessage) {
8383 * Fatal error: print error and exit
8484 */
8585 void RD_NORETURN fatal0 (const char *func, int line,
86 const char *fmt, ...) {
86 const char *fmt, ...) {
8787 va_list ap;
8888 char buf[1024];
8989
129129
130130 if (rkmessage->err) {
131131 KC_INFO(1, "Delivery failed for message: %s\n",
132 rd_kafka_err2str(rkmessage->err));
132 rd_kafka_err2str(rkmessage->err));
133133 stats.tx_err_dr++;
134134 return;
135135 }
136136
137137 KC_INFO(3, "Message delivered to partition %"PRId32" (offset %"PRId64")\n",
138 rkmessage->partition, rkmessage->offset);
138 rkmessage->partition, rkmessage->offset);
139139
140140 if (rkmessage->offset == 0 && say_once) {
141141 KC_INFO(3, "Enable message offset reporting "
142 "with '-X topic.produce.offset.report=true'\n");
142 "with '-X topic.produce.offset.report=true'\n");
143143 say_once = 0;
144144 }
145145 stats.tx_delivered++;
164164
165165 if (!conf.run)
166166 KC_FATAL("Program terminated while "
167 "producing message of %zd bytes", len);
167 "producing message of %zd bytes", len);
168168
169169 err = rd_kafka_producev(
170170 conf.rk,
183183
184184 if (err != RD_KAFKA_RESP_ERR__QUEUE_FULL)
185185 KC_FATAL("Failed to produce message (%zd bytes): %s",
186 len, rd_kafka_err2str(err));
186 len, rd_kafka_err2str(err));
187187
188188 stats.tx_err_q++;
189189
206206 int fd;
207207 void *ptr;
208208 struct stat st;
209 ssize_t sz;
210 int msgflags = 0;
209 ssize_t sz;
210 int msgflags = 0;
211211
212212 if ((fd = _COMPAT(open)(path, O_RDONLY)) == -1) {
213213 KC_INFO(1, "Failed to open %s: %s\n", path, strerror(errno));
233233 _COMPAT(close)(fd);
234234 return -1;
235235 }
236 sz = st.st_size;
237 msgflags = RD_KAFKA_MSG_F_COPY;
236 sz = st.st_size;
237 msgflags = RD_KAFKA_MSG_F_COPY;
238238 #else
239 ptr = malloc(st.st_size);
240 if (!ptr) {
241 KC_INFO(1, "Failed to allocate message for %s: %s\n",
242 path, strerror(errno));
243 _COMPAT(close)(fd);
244 return -1;
245 }
246
247 sz = _read(fd, ptr, st.st_size);
248 if (sz < st.st_size) {
249 KC_INFO(1, "Read failed for %s (%zd/%zd): %s\n",
250 path, sz, (size_t)st.st_size, sz == -1 ? strerror(errno) :
251 "incomplete read");
252 free(ptr);
253 close(fd);
254 return -1;
255 }
256 msgflags = RD_KAFKA_MSG_F_FREE;
239 ptr = malloc(st.st_size);
240 if (!ptr) {
241 KC_INFO(1, "Failed to allocate message for %s: %s\n",
242 path, strerror(errno));
243 _COMPAT(close)(fd);
244 return -1;
245 }
246
247 sz = _read(fd, ptr, st.st_size);
248 if (sz < st.st_size) {
249 KC_INFO(1, "Read failed for %s (%zd/%zd): %s\n",
250 path, sz, (size_t)st.st_size, sz == -1 ? strerror(errno) :
251 "incomplete read");
252 free(ptr);
253 close(fd);
254 return -1;
255 }
256 msgflags = RD_KAFKA_MSG_F_FREE;
257257 #endif
258258
259259 KC_INFO(4, "Producing file %s (%"PRIdMAX" bytes)\n",
260 path, (intmax_t)st.st_size);
261 produce(ptr, sz, NULL, 0, msgflags);
262
263 _COMPAT(close)(fd);
264
265 if (!(msgflags & RD_KAFKA_MSG_F_FREE)) {
260 path, (intmax_t)st.st_size);
261 produce(ptr, sz, NULL, 0, msgflags);
262
263 _COMPAT(close)(fd);
264
265 if (!(msgflags & RD_KAFKA_MSG_F_FREE)) {
266266 #ifndef _MSC_VER
267 munmap(ptr, st.st_size);
267 munmap(ptr, st.st_size);
268268 #else
269 free(ptr);
270 #endif
271 }
272 return sz;
269 free(ptr);
270 #endif
271 }
272 return sz;
273273 }
274274
275275
317317 conf.exitcode = 1;
318318 else if (good < pathcnt)
319319 KC_INFO(1, "Failed to produce from %i/%i files\n",
320 pathcnt - good, pathcnt);
320 pathcnt - good, pathcnt);
321321
322322 } else {
323323 /* Read messages from input, delimited by conf.delim */
392392 if (conf.flags & CONF_F_TEE &&
393393 fwrite(sbuf, orig_len, 1, stdout) != 1)
394394 KC_FATAL("Tee write error for message of %zd bytes: %s",
395 orig_len, strerror(errno));
395 orig_len, strerror(errno));
396396
397397 if (msgflags & RD_KAFKA_MSG_F_FREE) {
398398 /* rdkafka owns the allocated buffer
409409 if (conf.run) {
410410 if (!feof(fp))
411411 KC_FATAL("Unable to read message: %s",
412 strerror(errno));
412 strerror(errno));
413413 }
414414 }
415415
460460 }
461461
462462 KC_INFO(1, "Reached end of topic %s [%"PRId32"] "
463 "at offset %"PRId64"%s\n",
464 rd_kafka_topic_name(rkmessage->rkt),
465 rkmessage->partition,
466 rkmessage->offset,
467 !conf.run ? ": exiting" : "");
463 "at offset %"PRId64"%s\n",
464 rd_kafka_topic_name(rkmessage->rkt),
465 rkmessage->partition,
466 rkmessage->offset,
467 !conf.run ? ": exiting" : "");
468468 }
469469
470470
485485
486486 if (rkmessage->rkt)
487487 KC_FATAL("Topic %s [%"PRId32"] error: %s",
488 rd_kafka_topic_name(rkmessage->rkt),
489 rkmessage->partition,
490 rd_kafka_message_errstr(rkmessage));
488 rd_kafka_topic_name(rkmessage->rkt),
489 rkmessage->partition,
490 rd_kafka_message_errstr(rkmessage));
491491 else
492492 KC_FATAL("Consumer error: %s",
493 rd_kafka_message_errstr(rkmessage));
493 rd_kafka_message_errstr(rkmessage));
494494
495495 }
496496
499499 if (ts >= conf.stopts) {
500500 stop_partition(rkmessage);
501501 KC_INFO(1, "Reached stop timestamp for topic %s [%"PRId32"] "
502 "at offset %"PRId64"%s\n",
503 rd_kafka_topic_name(rkmessage->rkt),
504 rkmessage->partition,
505 rkmessage->offset,
506 !conf.run ? ": exiting" : "");
502 "at offset %"PRId64"%s\n",
503 rd_kafka_topic_name(rkmessage->rkt),
504 rkmessage->partition,
505 rkmessage->offset,
506 !conf.run ? ": exiting" : "");
507507 return;
508508 }
509509 }
528528 static void throttle_cb (rd_kafka_t *rk, const char *broker_name,
529529 int32_t broker_id, int throttle_time_ms, void *opaque){
530530 KC_INFO(1, "Broker %s (%"PRId32") throttled request for %dms\n",
531 broker_name, broker_id, throttle_time_ms);
531 broker_name, broker_id, throttle_time_ms);
532532 }
533533 #endif
534534
551551 void *opaque) {
552552
553553 KC_INFO(1, "Group %s rebalanced (memberid %s): ",
554 conf.group, rd_kafka_memberid(rk));
555
556 switch (err)
557 {
558 case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
554 conf.group, rd_kafka_memberid(rk));
555
556 switch (err)
557 {
558 case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
559559 if (conf.verbosity >= 1) {
560560 fprintf(stderr, "assigned: ");
561561 print_partition_list(1, partitions);
562562 }
563 rd_kafka_assign(rk, partitions);
564 break;
565
566 case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
563 rd_kafka_assign(rk, partitions);
564 break;
565
566 case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
567567 if (conf.verbosity >= 1) {
568568 fprintf(stderr, "revoked: ");
569569 print_partition_list(1, partitions);
570570 }
571 rd_kafka_assign(rk, NULL);
572 break;
573
574 default:
571 rd_kafka_assign(rk, NULL);
572 break;
573
574 default:
575575 KC_INFO(0, "failed: %s\n", rd_kafka_err2str(err));
576 break;
577 }
576 break;
577 }
578578 }
579579
580580 /**
613613 /* Subscribe */
614614 if ((err = rd_kafka_subscribe(conf.rk, topiclist)))
615615 KC_FATAL("Failed to subscribe to %d topics: %s\n",
616 topiclist->cnt, rd_kafka_err2str(err));
616 topiclist->cnt, rd_kafka_err2str(err));
617617
618618 rd_kafka_topic_partition_list_destroy(topiclist);
619619
650650 int64_t *offsets;
651651 rd_kafka_resp_err_t err;
652652 rd_kafka_topic_partition_list_t *rktparlistp =
653 rd_kafka_topic_partition_list_new(1);
653 rd_kafka_topic_partition_list_new(1);
654654
655655 for (i = 0 ; i < topic->partition_cnt ; i++) {
656656 int32_t partition = topic->partitions[i].id;
661661 continue;
662662
663663 rd_kafka_topic_partition_list_add(rktparlistp,
664 rd_kafka_topic_name(conf.rkt),
665 partition)->offset = conf.startts;
664 rd_kafka_topic_name(conf.rkt),
665 partition)->offset = conf.startts;
666666
667667 if (conf.partition != RD_KAFKA_PARTITION_UA)
668668 break;
721721 /* Query broker for topic + partition information. */
722722 if ((err = rd_kafka_metadata(conf.rk, 0, conf.rkt, &metadata, 5000)))
723723 KC_FATAL("Failed to query metadata for topic %s: %s",
724 rd_kafka_topic_name(conf.rkt), rd_kafka_err2str(err));
724 rd_kafka_topic_name(conf.rkt), rd_kafka_err2str(err));
725725
726726 /* Error handling */
727727 if (metadata->topic_cnt == 0)
728728 KC_FATAL("No such topic in cluster: %s",
729 rd_kafka_topic_name(conf.rkt));
729 rd_kafka_topic_name(conf.rkt));
730730
731731 if ((err = metadata->topics[0].err))
732732 KC_FATAL("Topic %s error: %s",
733 rd_kafka_topic_name(conf.rkt), rd_kafka_err2str(err));
733 rd_kafka_topic_name(conf.rkt), rd_kafka_err2str(err));
734734
735735 if (metadata->topics[0].partition_cnt == 0)
736736 KC_FATAL("Topic %s has no partitions",
737 rd_kafka_topic_name(conf.rkt));
737 rd_kafka_topic_name(conf.rkt));
738738
739739 /* If Exit-at-EOF is enabled, set up array to track EOF
740740 * state for each partition. */
741741 if (conf.exit_eof || conf.stopts) {
742742 part_stop = calloc(sizeof(*part_stop),
743 metadata->topics[0].partition_cnt);
743 metadata->topics[0].partition_cnt);
744744
745745 if (conf.partition != RD_KAFKA_PARTITION_UA)
746746 part_stop_thres = 1;
750750
751751 #if RD_KAFKA_VERSION >= 0x00090300
752752 if (conf.startts) {
753 offsets = get_offsets(&metadata->topics[0]);
753 offsets = get_offsets(&metadata->topics[0]);
754754 }
755755 #endif
756756
784784 if (conf.partition != RD_KAFKA_PARTITION_UA &&
785785 i == metadata->topics[0].partition_cnt)
786786 KC_FATAL("Topic %s (with partitions 0..%i): "
787 "partition %i does not exist",
788 rd_kafka_topic_name(conf.rkt),
789 metadata->topics[0].partition_cnt-1,
790 conf.partition);
787 "partition %i does not exist",
788 rd_kafka_topic_name(conf.rkt),
789 metadata->topics[0].partition_cnt-1,
790 conf.partition);
791791
792792
793793 /* Read messages from Kafka, write to 'fp'. */
808808 conf.partition != partition)
809809 continue;
810810
811 /* Dont stop already stopped partitions */
812 if (!part_stop || !part_stop[partition])
813 rd_kafka_consume_stop(conf.rkt, partition);
811 /* Dont stop already stopped partitions */
812 if (!part_stop || !part_stop[partition])
813 rd_kafka_consume_stop(conf.rkt, partition);
814814
815815 rd_kafka_consume_stop(conf.rkt, partition);
816816 }
12091209 rd_kafka_resp_err_t fatal_err;
12101210
12111211 fatal_err = rd_kafka_fatal_error(rk, fatal_errstr,
1212 sizeof(fatal_errstr));
1212 sizeof(fatal_errstr));
12131213 KC_INFO(0, "FATAL CLIENT ERROR: %s: %s: terminating\n",
12141214 rd_kafka_err2str(fatal_err), fatal_errstr);
12151215 conf.run = 0;
12161216
12171217 } else
12181218 #endif
1219 if (err == RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN) {
1220 KC_ERROR("%s: %s", rd_kafka_err2str(err),
1221 reason ? reason : "");
1222 } else {
1223 KC_INFO(1, "ERROR: %s: %s\n", rd_kafka_err2str(err),
1224 reason ? reason : "");
1225 }
1219 if (err == RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN) {
1220 KC_ERROR("%s: %s", rd_kafka_err2str(err),
1221 reason ? reason : "");
1222 } else {
1223 KC_INFO(1, "ERROR: %s: %s\n", rd_kafka_err2str(err),
1224 reason ? reason : "");
1225 }
12261226 }
12271227
12281228
18181818 else
18191819 conf.mode = 'P';
18201820 KC_INFO(1, "Auto-selecting %s mode (use -P or -C to override)\n",
1821 conf.mode == 'C' ? "Consumer":"Producer");
1821 conf.mode == 'C' ? "Consumer":"Producer");
18221822 }
18231823
18241824
19401940
19411941 } else if (conf.mode == 'P') {
19421942 conf.delim = parse_delim(delim);
1943 if (conf.flags & CONF_F_KEY_DELIM)
1944 conf.key_delim = parse_delim(key_delim);
1943 if (conf.flags & CONF_F_KEY_DELIM)
1944 conf.key_delim = parse_delim(key_delim);
19451945 }
19461946
19471947 /* Automatically enable API version requests if needed and
19591959
19601960 int main (int argc, char **argv) {
19611961 #ifdef SIGIO
1962 char tmp[16];
1962 char tmp[16];
19631963 #endif
19641964 FILE *in = stdin;
1965 struct timeval tv;
1965 struct timeval tv;
19661966 rd_kafka_topic_partition_list_t *rktparlist = NULL;
19671967
19681968 /* Certain Docker images don't have kafkacat as the entry point,
19821982 signal(SIGPIPE, term);
19831983 #endif
19841984
1985 /* Seed rng for random partitioner, jitter, etc. */
1986 rd_gettimeofday(&tv, NULL);
1987 srand(tv.tv_usec);
1985 /* Seed rng for random partitioner, jitter, etc. */
1986 rd_gettimeofday(&tv, NULL);
1987 srand(tv.tv_usec);
19881988
19891989 /* Create config containers */
19901990 conf.rk_conf = rd_kafka_conf_new();
9393 #define CONF_F_OFFSET 0x4 /* Print offsets */
9494 #define CONF_F_TEE 0x8 /* Tee output when producing */
9595 #define CONF_F_NULL 0x10 /* -Z: Send empty messages as NULL */
96 #define CONF_F_LINE 0x20 /* Read files in line mode when producing */
96 #define CONF_F_LINE 0x20 /* Read files in line mode when producing */
9797 #define CONF_F_APIVERREQ 0x40 /* Enable api.version.request=true */
9898 #define CONF_F_APIVERREQ_USER 0x80 /* User set api.version.request */
9999 #define CONF_F_NO_CONF_SEARCH 0x100 /* Disable default config file search */
150150
151151
152152 void RD_NORETURN fatal0 (const char *func, int line,
153 const char *fmt, ...);
153 const char *fmt, ...);
154154
155155 void error0 (int erroronexit, const char *func, int line,
156 const char *fmt, ...);
156 const char *fmt, ...);
157157
158158 #define KC_FATAL(.../*fmt*/) fatal0(__FUNCTION__, __LINE__, __VA_ARGS__)
159159
160160 #define KC_ERROR(.../*fmt*/) error0(conf.exitonerror, __FUNCTION__, __LINE__, __VA_ARGS__)
161161
162162 /* Info printout */
163 #define KC_INFO(VERBLVL,.../*fmt*/) do { \
164 if (conf.verbosity >= (VERBLVL)) \
165 fprintf(stderr, "%% " __VA_ARGS__); \
163 #define KC_INFO(VERBLVL,.../*fmt*/) do { \
164 if (conf.verbosity >= (VERBLVL)) \
165 fprintf(stderr, "%% " __VA_ARGS__); \
166166 } while (0)
167167
168168
4040 */
4141
4242 #ifdef __FreeBSD__
43 #include <sys/endian.h>
43 #include <sys/endian.h>
4444 #elif defined __GLIBC__
45 #include <endian.h>
46 #ifndef be64toh
47 /* Support older glibc (<2.9) which lack be64toh */
48 #include <byteswap.h>
49 #if __BYTE_ORDER == __BIG_ENDIAN
50 #define be16toh(x) (x)
51 #define be32toh(x) (x)
52 #define be64toh(x) (x)
53 #define le64toh(x) __bswap_64 (x)
54 #define le32toh(x) __bswap_32 (x)
55 #else
56 #define be16toh(x) __bswap_16 (x)
57 #define be32toh(x) __bswap_32 (x)
58 #define be64toh(x) __bswap_64 (x)
59 #define le64toh(x) (x)
60 #define le32toh(x) (x)
61 #endif
62 #endif
45 #include <endian.h>
46 #ifndef be64toh
47 /* Support older glibc (<2.9) which lack be64toh */
48 #include <byteswap.h>
49 #if __BYTE_ORDER == __BIG_ENDIAN
50 #define be16toh(x) (x)
51 #define be32toh(x) (x)
52 #define be64toh(x) (x)
53 #define le64toh(x) __bswap_64 (x)
54 #define le32toh(x) __bswap_32 (x)
55 #else
56 #define be16toh(x) __bswap_16 (x)
57 #define be32toh(x) __bswap_32 (x)
58 #define be64toh(x) __bswap_64 (x)
59 #define le64toh(x) (x)
60 #define le32toh(x) (x)
61 #endif
62 #endif
6363
6464 #elif defined __CYGWIN__
65 #include <endian.h>
65 #include <endian.h>
6666 #elif defined __BSD__
67 #include <sys/endian.h>
67 #include <sys/endian.h>
6868 #elif defined __sun
69 #include <sys/byteorder.h>
70 #include <sys/isa_defs.h>
69 #include <sys/byteorder.h>
70 #include <sys/isa_defs.h>
7171 #define __LITTLE_ENDIAN 1234
7272 #define __BIG_ENDIAN 4321
7373 #ifdef _BIG_ENDIAN
9191 #endif /* __sun */
9292
9393 #elif defined __APPLE__
94 #include <machine/endian.h>
95 #include <libkern/OSByteOrder.h>
94 #include <machine/endian.h>
95 #include <libkern/OSByteOrder.h>
9696 #if __DARWIN_BYTE_ORDER == __DARWIN_BIG_ENDIAN
9797 #define be64toh(x) (x)
9898 #define be32toh(x) (x)
128128 (((x) & 0xff00) << 8) | \
129129 (((x) & 0xff0000) >> 8) | \
130130 (((x) & 0xff000000) >> 24))
131 #define le64toh(x) \
132 ((((x) & 0x00000000000000ffL) << 56) | \
133 (((x) & 0x000000000000ff00L) << 40) | \
134 (((x) & 0x0000000000ff0000L) << 24) | \
135 (((x) & 0x00000000ff000000L) << 8) | \
136 (((x) & 0x000000ff00000000L) >> 8) | \
137 (((x) & 0x0000ff0000000000L) >> 24) | \
138 (((x) & 0x00ff000000000000L) >> 40) | \
131 #define le64toh(x) \
132 ((((x) & 0x00000000000000ffL) << 56) | \
133 (((x) & 0x000000000000ff00L) << 40) | \
134 (((x) & 0x0000000000ff0000L) << 24) | \
135 (((x) & 0x00000000ff000000L) << 8) | \
136 (((x) & 0x000000ff00000000L) >> 8) | \
137 (((x) & 0x0000ff0000000000L) >> 24) | \
138 (((x) & 0x00ff000000000000L) >> 40) | \
139139 (((x) & 0xff00000000000000L) >> 56))
140140 #else
141 #include <endian.h>
141 #include <endian.h>
142142 #endif
143143
144144
0 /*
1 * librdkafka - Apache Kafka C library
2 *
3 * Copyright (c) 2012-2019 Magnus Edenhill
4 * All rights reserved.
5 *
6 * Redistribution and use in source and binary forms, with or without
7 * modification, are permitted provided that the following conditions are met:
8 *
9 * 1. Redistributions of source code must retain the above copyright notice,
10 * this list of conditions and the following disclaimer.
11 * 2. Redistributions in binary form must reproduce the above copyright notice,
12 * this list of conditions and the following disclaimer in the documentation
13 * and/or other materials provided with the distribution.
14 *
15 * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
16 * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
17 * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
18 * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
19 * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
20 * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
21 * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
22 * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
23 * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
24 * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
25 * POSSIBILITY OF SUCH DAMAGE.
26 */
27
028 #pragma once
129
230 /**
2048 typedef SSIZE_T ssize_t;
2149
2250 ssize_t getdelim (char **bufptr, size_t *n,
23 int delim, FILE *fp);
51 int delim, FILE *fp);
2452
2553
2654 /**
27 * @brief gettimeofday() for win32
28 */
55 * @brief gettimeofday() for win32
56 */
2957 static RD_UNUSED
3058 int rd_gettimeofday (struct timeval *tv, struct timezone *tz) {
31 SYSTEMTIME st;
32 FILETIME ft;
33 ULARGE_INTEGER d;
59 SYSTEMTIME st;
60 FILETIME ft;
61 ULARGE_INTEGER d;
3462
35 GetSystemTime(&st);
36 SystemTimeToFileTime(&st, &ft);
37 d.HighPart = ft.dwHighDateTime;
38 d.LowPart = ft.dwLowDateTime;
39 tv->tv_sec = (long)((d.QuadPart - 116444736000000000llu) / 10000000L);
40 tv->tv_usec = (long)(st.wMilliseconds * 1000);
63 GetSystemTime(&st);
64 SystemTimeToFileTime(&st, &ft);
65 d.HighPart = ft.dwHighDateTime;
66 d.LowPart = ft.dwLowDateTime;
67 tv->tv_sec = (long)((d.QuadPart - 116444736000000000llu) / 10000000L);
68 tv->tv_usec = (long)(st.wMilliseconds * 1000);
4169
42 return 0;
70 return 0;
4371 }
4472
4573
5280 #define _COMPAT(FUNC) FUNC
5381
5482 #define rd_gettimeofday(tv,tz) gettimeofday(tv,tz)
55 #endif
83 #endif