Skip to content

Remove incremental updates #425

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 2 commits into from
Jul 19, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
60 changes: 5 additions & 55 deletions confluent_kafka/admin/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -103,27 +103,14 @@ class Type(Enum):
BROKER = RESOURCE_BROKER #: Broker resource. Resource name is broker id

def __init__(self, restype, name,
set_config=None, add_config=None, del_config=None,
described_configs=None, error=None):
set_config=None, described_configs=None, error=None):
"""
:param ConfigResource.Type restype: Resource type.
:param str name: Resource name, depending on restype.
For RESOURCE_BROKER the resource name is the broker id.
:param dict set_config: Configuration to set/overwrite. Dict of str, str.
:param dict add_config: Configuration to add/append. Dict of str, str.
Requires broker version with KIP-248 support.
:param list del_config: Configuration to delete/revert to default. List of str.
Requires broker version with KIP-248 support.
:param dict described_configs: For internal use only.
:param KafkaError error: For internal use only.

When alter_configs(incremental=False) only set_config is permitted,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't remove this sentence, it still applies, just remove the incremental stuff.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The other methods have been removed as part of this change so there is no need to retain this message

and any configuration parameter not specified will be reverted to
its default value.

With alter_configs(incremental=True) (requires broker version with KIP-248 support)
only the configuration parameters specified through set, add or del
will be modified.
"""
super(ConfigResource, self).__init__()

Expand All @@ -150,16 +137,6 @@ def __init__(self, restype, name,
else:
self.set_config_dict = dict()

if add_config is not None:
self.add_config_dict = add_config.copy()
else:
self.add_config_dict = dict()

if del_config is not None:
self.del_config_dict = dict((k, None) for k in del_config)
else:
self.del_config_dict = dict()

self.configs = described_configs
self.error = error

Expand All @@ -185,16 +162,15 @@ def __len__(self):
:rtype: int
:returns: number of configuration entries/operations
"""
return len(self.add_config_dict) + len(self.set_config_dict) + len(self.del_config_dict)
return len(self.set_config_dict)

def set_config(self, name, value, overwrite=True):
"""
Set/Overwrite configuration entry

Unless alter_configs() is called with `incremental=True` any configuration properties
that are not included will be reverted to their default values.
As a workaround use describe_configs() to retrieve the current
configuration and overwrite the settings you want to change.
Any configuration properties that are not included will be reverted to their default values.
As a workaround use describe_configs() to retrieve the current configuration and
overwrite the settings you want to change.

:param str name: Configuration property name
:param str value: Configuration value
Expand All @@ -205,27 +181,6 @@ def set_config(self, name, value, overwrite=True):
return
self.set_config_dict[name] = value

def add_config(self, name, value):
"""
Append value to configuration entry.

Requires broker version with KIP-248 support and alter_configs(.., incremental=True).

:param str name: Configuration property name
:param str value: Configuration value
"""
self.add_config_dict[name] = value

def del_config(self, name):
"""
Delete configuration entry, reverting it to the default value.

Requires broker version with KIP-248 support and alter_configs(.., incremental=True).

:param str name: Configuration property name
"""
self.del_config_dict[name] = None


class AdminClient (_AdminClientImpl):
"""
Expand Down Expand Up @@ -478,8 +433,6 @@ def alter_configs(self, resources, **kwargs):
the provided resources with the new configuration given,
reverting all other configuration for the resource back
to their default values.
Use incremental=True to change the behaviour so that only the
passed configuration is modified, requires broker version with KIP-248 support.

:warning: Multiple resources and resource types may be specified,
but at most one resource of type RESOURCE_BROKER is allowed
Expand All @@ -492,9 +445,6 @@ def alter_configs(self, resources, **kwargs):
on broker, and response. Default: `socket.timeout.ms*1000.0`.
:param bool validate_only: Tell broker to only validate the request,
without altering the configuration. Default: False
:param bool incremental: If true, only update the specified configuration
entries, not reverting unspecified configuration.
This requires broker version with KIP-248 support. Default: False

:returns: a dict of futures for each resource, keyed by the ConfigResource.
:rtype: dict(<ConfigResource, future>)
Expand Down
122 changes: 37 additions & 85 deletions confluent_kafka/src/Admin.c
Original file line number Diff line number Diff line change
Expand Up @@ -75,15 +75,13 @@ struct Admin_options {
float request_timeout; /* parser: f */
float operation_timeout; /* parser: f */
int broker; /* parser: i */
int incremental; /* needs special bool parsing */
};

/**@brief "unset" value initializers for Admin_options
* Make sure this is kept up to date with Admin_options above. */
#define Admin_options_INITIALIZER { \
Admin_options_def_int, Admin_options_def_float, \
Admin_options_def_float, Admin_options_def_int, \
Admin_options_def_int, \
}

#define Admin_options_is_set_int(v) ((v) != Admin_options_def_int)
Expand Down Expand Up @@ -143,12 +141,6 @@ Admin_options_to_c (Handle *self, rd_kafka_admin_op_t for_api,
errstr, sizeof(errstr))))
goto err;

if (Admin_options_is_set_int(options->incremental) &&
(err = rd_kafka_AdminOptions_set_incremental(
c_options, options->incremental,
errstr, sizeof(errstr))))
goto err;

return c_options;

err:
Expand Down Expand Up @@ -250,7 +242,7 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
}

/**
* @brief Translate a dict to ConfigResource {set,add,delete}_config() calls,
* @brief Translate a dict to ConfigResource set_config() calls,
* or to NewTopic_add_config() calls.
*
*
Expand All @@ -276,52 +268,44 @@ Admin_config_dict_to_c (void *c_obj, PyObject *dict, const char *op_name) {

k = cfl_PyUnistr_AsUTF8(ks, &ks8);

if (!strcmp(op_name, "del_config")) {
err = rd_kafka_ConfigResource_delete_config(
(rd_kafka_ConfigResource_t *)c_obj, k);
} else {
PyObject *vs = NULL, *vs8 = NULL;
if (!(vs = cfl_PyObject_Unistr(vo)) ||
!(v = cfl_PyUnistr_AsUTF8(vs, &vs8))) {
PyErr_Format(PyExc_ValueError,
"expect %s config value for %s "
"to be unicode string",
op_name, k);
Py_XDECREF(vs);
Py_XDECREF(vs8);
Py_DECREF(ks);
Py_XDECREF(ks8);
return 0;
}

if (!strcmp(op_name, "add_config"))
err = rd_kafka_ConfigResource_add_config(
(rd_kafka_ConfigResource_t *)c_obj,
k, v);
else if (!strcmp(op_name, "set_config"))
err = rd_kafka_ConfigResource_set_config(
(rd_kafka_ConfigResource_t *)c_obj,
k, v);
else if (!strcmp(op_name, "newtopic_set_config"))
err = rd_kafka_NewTopic_set_config(
(rd_kafka_NewTopic_t *)c_obj, k, v);
else
err = RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED;

if (err) {
PyErr_Format(PyExc_ValueError,
"%s config %s failed: %s",
op_name, k, rd_kafka_err2str(err));
Py_XDECREF(vs);
Py_XDECREF(vs8);
Py_DECREF(ks);
Py_XDECREF(ks8);
return 0;
}
PyObject *vs = NULL, *vs8 = NULL;
if (!(vs = cfl_PyObject_Unistr(vo)) ||
!(v = cfl_PyUnistr_AsUTF8(vs, &vs8))) {
PyErr_Format(PyExc_ValueError,
"expect %s config value for %s "
"to be unicode string",
op_name, k);
Py_XDECREF(vs);
Py_XDECREF(vs8);
Py_DECREF(ks);
Py_XDECREF(ks8);
return 0;
}

if (!strcmp(op_name, "set_config"))
err = rd_kafka_ConfigResource_set_config(
(rd_kafka_ConfigResource_t *)c_obj,
k, v);
else if (!strcmp(op_name, "newtopic_set_config"))
err = rd_kafka_NewTopic_set_config(
(rd_kafka_NewTopic_t *)c_obj, k, v);
else
err = RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED;

if (err) {
PyErr_Format(PyExc_ValueError,
"%s config %s failed: %s",
op_name, k, rd_kafka_err2str(err));
Py_XDECREF(vs);
Py_XDECREF(vs8);
Py_DECREF(ks);
Py_XDECREF(ks8);
return 0;
}

Py_XDECREF(vs);
Py_XDECREF(vs8);
Py_DECREF(ks);
Py_XDECREF(ks8);
}
Expand Down Expand Up @@ -835,14 +819,13 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
PyObject *kwargs) {
PyObject *resources, *future;
PyObject *validate_only_obj = NULL, *incremental_obj = NULL;
PyObject *validate_only_obj = NULL;
static char *kws[] = { "resources",
"future",
/* options */
"validate_only",
"request_timeout",
"broker",
"incremental",
NULL };
struct Admin_options options = Admin_options_INITIALIZER;
rd_kafka_AdminOptions_t *c_options = NULL;
Expand All @@ -853,12 +836,11 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
CallState cs;

/* topics is a list of NewPartitions_t objects. */
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "OO|OfiO", kws,
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "OO|Ofi", kws,
&resources, &future,
&validate_only_obj,
&options.request_timeout,
&options.broker,
&incremental_obj))
&options.broker))
return NULL;

if (!PyList_Check(resources) ||
Expand All @@ -874,12 +856,6 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
&options.validate_only))
return NULL;

if (incremental_obj &&
!cfl_PyBool_get(incremental_obj, "incremental",
&options.incremental))
return NULL;


c_options = Admin_options_to_c(self, RD_KAFKA_ADMIN_OP_ALTERCONFIGS,
&options, future);
if (!c_options)
Expand Down Expand Up @@ -955,30 +931,6 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
goto err;
}
Py_DECREF(dict);

if (!cfl_PyObject_GetAttr(res, "add_config_dict", &dict,
&PyDict_Type, 1)) {
i++;
goto err;
}
if (!Admin_config_dict_to_c(c_objs[i], dict, "add_config")) {
Py_DECREF(dict);
i++;
goto err;
}
Py_DECREF(dict);

if (!cfl_PyObject_GetAttr(res, "del_config_dict", &dict,
&PyDict_Type, 1)) {
i++;
goto err;
}
if (!Admin_config_dict_to_c(c_objs[i], dict, "del_config")) {
Py_DECREF(dict);
i++;
goto err;
}
Py_DECREF(dict);
}


Expand Down
7 changes: 2 additions & 5 deletions examples/adminapi.py
Original file line number Diff line number Diff line change
Expand Up @@ -143,11 +143,8 @@ def example_alter_configs(a, args):

def example_delta_alter_configs(a, args):
"""
Alter only supplied configs (pre incremental/KIP-248)
The pre incremental/KIP-248 AlterConfigs Kafka API requires all
configuration to be passed, any left out configuration properties will
revert to their default settings.
The AlterConfigs Kafka API requires all configuration to be passed,
any left out configuration properties will revert to their default settings.
This example shows how to just modify the supplied configuration entries
by first reading the configuration from the broker, updating the supplied
Expand Down
8 changes: 1 addition & 7 deletions tests/test_Admin.py
Original file line number Diff line number Diff line change
Expand Up @@ -244,15 +244,9 @@ def test_alter_configs_api():
with pytest.raises(ValueError):
a.alter_configs([])

with pytest.raises(ValueError):
a.alter_configs([None, ConfigResource("topic", "mytopic",
add_config={"something": "else"})])

fs = a.alter_configs([ConfigResource("topic", "mytopic",
set_config={"set": "this",
"and": "this"},
add_config={"add": "this"},
del_config=["this"]),
"and": "this"}),
ConfigResource(confluent_kafka.admin.RESOURCE_GROUP,
"mygroup")],
request_timeout=0.123)
Expand Down
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy