Skip to content

Commit bd98161

Browse files
committed
Add alter configs operation
1 parent 0d3db05 commit bd98161

File tree

2 files changed

+192
-0
lines changed

2 files changed

+192
-0
lines changed
Lines changed: 115 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,115 @@
1+
/*
2+
* Copyright (c) 2018, Matias Fontanini
3+
* All rights reserved.
4+
*
5+
* Redistribution and use in source and binary forms, with or without
6+
* modification, are permitted provided that the following conditions are
7+
* met:
8+
*
9+
* * Redistributions of source code must retain the above copyright
10+
* notice, this list of conditions and the following disclaimer.
11+
* * Redistributions in binary form must reproduce the above
12+
* copyright notice, this list of conditions and the following disclaimer
13+
* in the documentation and/or other materials provided with the
14+
* distribution.
15+
*
16+
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
17+
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
18+
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
19+
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
20+
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
21+
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
22+
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
23+
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
24+
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
25+
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
26+
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
27+
*
28+
*/
29+
30+
#ifndef CPPKAFKA_ADMIN_ALTER_CONFIGS_OPERATION_H
31+
#define CPPKAFKA_ADMIN_ALTER_CONFIGS_OPERATION_H
32+
33+
#include <memory>
34+
#include "compound_operation.h"
35+
#include "operation.h"
36+
#include "../configuration_option.h"
37+
38+
#if RD_KAFKA_VERSION >= RD_KAFKA_ADMIN_API_SUPPORT_VERSION
39+
40+
namespace cppkafka {
41+
namespace admin {
42+
43+
/**
44+
* \brief Modifies the configuration for a cluster, broker or topic
45+
*/
46+
class AlterConfigsOperation : public Operation {
47+
public:
48+
/**
49+
* The rdkafka type used for this operation's handle
50+
*/
51+
using HandleType = rd_kafka_ConfigResource_t;
52+
53+
/**
54+
* \brief Represents the resource type being modified
55+
*/
56+
enum class ResourceType {
57+
Unknown = RD_KAFKA_RESOURCE_UNKNOWN,
58+
Any = RD_KAFKA_RESOURCE_ANY,
59+
Topic = RD_KAFKA_RESOURCE_TOPIC,
60+
Group = RD_KAFKA_RESOURCE_GROUP,
61+
Broker = RD_KAFKA_RESOURCE_BROKER
62+
};
63+
64+
/**
65+
* \brief Constructs an alter configs operation
66+
*
67+
* The name depends on the type being used. For example, when modifying a topic
68+
* type, this will be the name of the topic to be modifies.
69+
*
70+
* See KIP-133 for more information.
71+
*
72+
* \param type The type of the resource to be modified
73+
* \param name The name of the resource to be modified
74+
*/
75+
AlterConfigsOperation(ResourceType type, const std::string& name);
76+
77+
/**
78+
* \brief Sets a config option
79+
*
80+
* \param config_option The configuration option to be set
81+
*/
82+
void set_config(const ConfigurationOption& config_option);
83+
84+
/**
85+
* \brief Removes a configuration option.
86+
*
87+
* Note that this implies removing it from the server side, not on list
88+
* of configs added to this alter configs operation via AlterConfigsOperation::set_config
89+
*
90+
* \param key The config key to be removed
91+
*/
92+
void remove_config(const std::string& key);
93+
private:
94+
using HandlePtr = std::unique_ptr<HandleType,
95+
decltype(&rd_kafka_ConfigResource_destroy)>;
96+
97+
void do_execute(KafkaHandleBase& kafka_handle,
98+
Queue& queue,
99+
const OperationOptions* options) override;
100+
101+
HandlePtr handle_;
102+
};
103+
104+
/**
105+
* \brief Alters the config for multiple resources
106+
*/
107+
using CompoundAlterConfigsOperation = CompoundOperation<AlterConfigsOperation,
108+
&rd_kafka_AlterConfigs>;
109+
110+
} // admin
111+
} // cppkafka
112+
113+
#endif // Admin API
114+
115+
#endif // CPPKAFKA_ADMIN_ALTER_CONFIGS_OPERATION_H

src/admin/alter_configs_operation.cpp

Lines changed: 77 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,77 @@
1+
/*
2+
* Copyright (c) 2018, Matias Fontanini
3+
* All rights reserved.
4+
*
5+
* Redistribution and use in source and binary forms, with or without
6+
* modification, are permitted provided that the following conditions are
7+
* met:
8+
*
9+
* * Redistributions of source code must retain the above copyright
10+
* notice, this list of conditions and the following disclaimer.
11+
* * Redistributions in binary form must reproduce the above
12+
* copyright notice, this list of conditions and the following disclaimer
13+
* in the documentation and/or other materials provided with the
14+
* distribution.
15+
*
16+
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
17+
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
18+
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
19+
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
20+
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
21+
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
22+
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
23+
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
24+
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
25+
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
26+
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
27+
*
28+
*/
29+
30+
#include "admin/alter_configs_operation.h"
31+
#include "exceptions.h"
32+
33+
using std::string;
34+
35+
#if RD_KAFKA_VERSION >= RD_KAFKA_ADMIN_API_SUPPORT_VERSION
36+
37+
namespace cppkafka {
38+
namespace admin {
39+
40+
AlterConfigsOperation::AlterConfigsOperation(ResourceType type, const string& name)
41+
: handle_(nullptr, nullptr) {
42+
auto ptr = rd_kafka_ConfigResource_new(static_cast<rd_kafka_ResourceType_t>(type),
43+
name.data());
44+
if (!ptr) {
45+
throw AdminOperationException("Failed to create alter config handle");
46+
}
47+
handle_ = HandlePtr(ptr, rd_kafka_ConfigResource_destroy);
48+
}
49+
50+
void AlterConfigsOperation::set_config(const ConfigurationOption& config_option) {
51+
const string& key = config_option.get_key();
52+
const string& value = config_option.get_value();
53+
const Error result = rd_kafka_ConfigResource_set_config(handle_.get(),
54+
key.data(),
55+
value.data());
56+
if (!result) {
57+
throw AdminOperationException(result.to_string());
58+
}
59+
}
60+
61+
void AlterConfigsOperation::remove_config(const string& key) {
62+
const Error result = rd_kafka_ConfigResource_set_config(handle_.get(), key.data(), nullptr);
63+
if (!result) {
64+
throw AdminOperationException(result.to_string());
65+
}
66+
}
67+
68+
void AlterConfigsOperation::do_execute(KafkaHandleBase& kafka_handle,
69+
Queue& queue,
70+
const OperationOptions* options) {
71+
run_operation(kafka_handle, queue, options, handle_.get(), &rd_kafka_AlterConfigs);
72+
}
73+
74+
} // admin
75+
} // cppkafka
76+
77+
#endif // Admin API

0 commit comments

Comments
 (0)
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