diff --git a/CONFIGURATION.md b/CONFIGURATION.md
index 147ddfa703..6d0390996c 100644
--- a/CONFIGURATION.md
+++ b/CONFIGURATION.md
@@ -95,7 +95,7 @@ max.partition.fetch.bytes | C | |
fetch.max.bytes | C | 0 .. 2147483135 | 52428800 | medium | Maximum amount of data the broker shall return for a Fetch request. Messages are fetched in batches by the consumer and if the first message batch in the first non-empty partition of the Fetch request is larger than this value, then the message batch will still be returned to ensure the consumer can make progress. The maximum message batch size accepted by the broker is defined via `message.max.bytes` (broker config) or `max.message.bytes` (broker topic config). `fetch.max.bytes` is automatically adjusted upwards to be at least `message.max.bytes` (consumer config).
*Type: integer*
fetch.min.bytes | C | 1 .. 100000000 | 1 | low | Minimum number of bytes the broker responds with. If fetch.wait.max.ms expires the accumulated data will be sent to the client regardless of this setting.
*Type: integer*
fetch.error.backoff.ms | C | 0 .. 300000 | 500 | medium | How long to postpone the next fetch request for a topic+partition in case of a fetch error.
*Type: integer*
-offset.store.method | C | none, file, broker | broker | low | Offset commit store method: 'file' - local file store (offset.store.path, et.al), 'broker' - broker commit store (requires Apache Kafka 0.8.2 or later on the broker).
*Type: enum value*
+offset.store.method | C | none, file, broker | broker | low | **DEPRECATED** Offset commit store method: 'file' - DEPRECATED: local file store (offset.store.path, et.al), 'broker' - broker commit store (requires Apache Kafka 0.8.2 or later on the broker).
*Type: enum value*
consume_cb | C | | | low | Message consume callback (set with rd_kafka_conf_set_consume_cb())
*Type: pointer*
rebalance_cb | C | | | low | Called after consumer group has been rebalanced (set with rd_kafka_conf_set_rebalance_cb())
*Type: pointer*
offset_commit_cb | C | | | low | Offset commit result propagation callback. (set with rd_kafka_conf_set_offset_commit_cb())
*Type: pointer*
@@ -141,9 +141,9 @@ auto.commit.enable | C | true, false | true
enable.auto.commit | C | | | low | Alias for `auto.commit.enable`
auto.commit.interval.ms | C | 10 .. 86400000 | 60000 | high | [**LEGACY PROPERTY:** This setting is used by the simple legacy consumer only. When using the high-level KafkaConsumer, the global `auto.commit.interval.ms` property must be used instead]. The frequency in milliseconds that the consumer offsets are committed (written) to offset storage.
*Type: integer*
auto.offset.reset | C | smallest, earliest, beginning, largest, latest, end, error | largest | high | Action to take when there is no initial offset in offset store or the desired offset is out of range: 'smallest','earliest' - automatically reset the offset to the smallest offset, 'largest','latest' - automatically reset the offset to the largest offset, 'error' - trigger an error which is retrieved by consuming messages and checking 'message->err'.
*Type: enum value*
-offset.store.path | C | | . | low | Path to local file for storing offsets. If the path is a directory a filename will be automatically generated in that directory based on the topic and partition.
*Type: string*
-offset.store.sync.interval.ms | C | -1 .. 86400000 | -1 | low | fsync() interval for the offset file, in milliseconds. Use -1 to disable syncing, and 0 for immediate sync after each write.
*Type: integer*
-offset.store.method | C | file, broker | broker | low | Offset commit store method: 'file' - local file store (offset.store.path, et.al), 'broker' - broker commit store (requires "group.id" to be configured and Apache Kafka 0.8.2 or later on the broker.).
*Type: enum value*
+offset.store.path | C | | . | low | **DEPRECATED** Path to local file for storing offsets. If the path is a directory a filename will be automatically generated in that directory based on the topic and partition. File-based offset storage will be removed in a future version.
*Type: string*
+offset.store.sync.interval.ms | C | -1 .. 86400000 | -1 | low | **DEPRECATED** fsync() interval for the offset file, in milliseconds. Use -1 to disable syncing, and 0 for immediate sync after each write. File-based offset storage will be removed in a future version.
*Type: integer*
+offset.store.method | C | file, broker | broker | low | **DEPRECATED** Offset commit store method: 'file' - DEPRECATED: local file store (offset.store.path, et.al), 'broker' - broker commit store (requires "group.id" to be configured and Apache Kafka 0.8.2 or later on the broker.).
*Type: enum value*
consume.callback.max.messages | C | 0 .. 1000000 | 0 | low | Maximum number of messages to dispatch in one `rd_kafka_consume_callback*()` call (0 = unlimited)
*Type: integer*
### C/P legend: C = Consumer, P = Producer, * = both
diff --git a/src/rdkafka.h b/src/rdkafka.h
index f143812de0..35131adccc 100644
--- a/src/rdkafka.h
+++ b/src/rdkafka.h
@@ -1822,9 +1822,17 @@ void *rd_kafka_opaque(const rd_kafka_t *rk);
/**
- * Sets the default topic configuration to use for automatically
- * subscribed topics (e.g., through pattern-matched topics).
- * The topic config object is not usable after this call.
+ * @brief Sets the default topic configuration to use for automatically
+ * subscribed topics (e.g., through pattern-matched topics).
+ * The topic config object is not usable after this call.
+ *
+ * @warning Any topic configuration settings that have been set on the
+ * global rd_kafka_conf_t object will be overwritten by this call
+ * since the implicitly created default topic config object is
+ * replaced by the user-supplied one.
+ *
+ * @deprecated Set default topic level configuration on the
+ * global rd_kafka_conf_t object instead.
*/
RD_EXPORT
void rd_kafka_conf_set_default_topic_conf (rd_kafka_conf_t *conf,
@@ -3753,6 +3761,8 @@ void rd_kafka_set_logger(rd_kafka_t *rk,
*
* If the \p \"debug\" configuration property is set the level is automatically
* adjusted to \c LOG_DEBUG (7).
+ *
+ * @deprecated Set the \c "log_level" configuration property instead.
*/
RD_EXPORT
void rd_kafka_set_log_level(rd_kafka_t *rk, int level);
diff --git a/src/rdkafka_conf.c b/src/rdkafka_conf.c
index e32e14f5d1..9e3a23e074 100644
--- a/src/rdkafka_conf.c
+++ b/src/rdkafka_conf.c
@@ -855,10 +855,11 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"How long to postpone the next fetch request for a "
"topic+partition in case of a fetch error.",
0, 300*1000, 500 },
- { _RK_GLOBAL|_RK_CONSUMER, "offset.store.method", _RK_C_S2I,
+ { _RK_GLOBAL|_RK_CONSUMER|_RK_DEPRECATED, "offset.store.method",
+ _RK_C_S2I,
_RK(offset_store_method),
"Offset commit store method: "
- "'file' - local file store (offset.store.path, et.al), "
+ "'file' - DEPRECATED: local file store (offset.store.path, et.al), "
"'broker' - broker commit store "
"(requires Apache Kafka 0.8.2 or later on the broker).",
.vdef = RD_KAFKA_OFFSET_METHOD_BROKER,
@@ -1162,24 +1163,29 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
{ RD_KAFKA_OFFSET_INVALID, "error" },
}
},
- { _RK_TOPIC|_RK_CONSUMER, "offset.store.path", _RK_C_STR,
+ { _RK_TOPIC|_RK_CONSUMER|_RK_DEPRECATED, "offset.store.path",
+ _RK_C_STR,
_RKT(offset_store_path),
"Path to local file for storing offsets. If the path is a directory "
"a filename will be automatically generated in that directory based "
- "on the topic and partition.",
+ "on the topic and partition. "
+ "File-based offset storage will be removed in a future version.",
.sdef = "." },
- { _RK_TOPIC|_RK_CONSUMER, "offset.store.sync.interval.ms", _RK_C_INT,
+ { _RK_TOPIC|_RK_CONSUMER|_RK_DEPRECATED,
+ "offset.store.sync.interval.ms", _RK_C_INT,
_RKT(offset_store_sync_interval_ms),
"fsync() interval for the offset file, in milliseconds. "
"Use -1 to disable syncing, and 0 for immediate sync after "
- "each write.",
+ "each write. "
+ "File-based offset storage will be removed in a future version.",
-1, 86400*1000, -1 },
- { _RK_TOPIC|_RK_CONSUMER, "offset.store.method", _RK_C_S2I,
+ { _RK_TOPIC|_RK_CONSUMER|_RK_DEPRECATED, "offset.store.method",
+ _RK_C_S2I,
_RKT(offset_store_method),
"Offset commit store method: "
- "'file' - local file store (offset.store.path, et.al), "
+ "'file' - DEPRECATED: local file store (offset.store.path, et.al), "
"'broker' - broker commit store "
"(requires \"group.id\" to be configured and "
"Apache Kafka 0.8.2 or later on the broker.).",