Skip to content

Commit

Permalink
[improve][cpp-client] Add basic authentication
Browse files Browse the repository at this point in the history
Signed-off-by: Zixuan Liu <[email protected]>
  • Loading branch information
nodece committed Jun 6, 2022
1 parent 91cb00e commit 9f556d3
Show file tree
Hide file tree
Showing 9 changed files with 274 additions and 1 deletion.
46 changes: 46 additions & 0 deletions pulsar-client-cpp/include/pulsar/Authentication.h
Original file line number Diff line number Diff line change
Expand Up @@ -293,6 +293,52 @@ class PULSAR_PUBLIC AuthToken : public Authentication {
AuthenticationDataPtr authDataToken_;
};

/**
* Basic based implementation of Pulsar client authentication
*/
class PULSAR_PUBLIC AuthBasic : public Authentication {
public:
explicit AuthBasic(AuthenticationDataPtr&);
~AuthBasic() override;

/**
* Create an AuthBasic with a ParamMap
*
* It is equal to create(params[“username”], params[“password”])
* @see create(const std::string&, const std::string&)
*/
static AuthenticationPtr create(ParamMap& params);

/**
* Create an AuthBasic with an authentication parameter string
*
* @param authParamsString the JSON format string: {"username": "admin", "password": "123456"}
*/
static AuthenticationPtr create(const std::string& authParamsString);

/**
* Create an AuthBasic with the required parameters
*/
static AuthenticationPtr create(const std::string& username, const std::string& password);

/**
* @return “basic”
*/
const std::string getAuthMethodName() const override;

/**
* Get AuthenticationData from the current instance
*
* @param[out] authDataBasic the shared pointer of AuthenticationData. The content of AuthenticationData
* is changed to the internal data of the current instance.
* @return ResultOk
*/
Result getAuthData(AuthenticationDataPtr& authDataBasic) override;

private:
AuthenticationDataPtr authDataBasic_;
};

/**
* Athenz implementation of Pulsar client authentication
*/
Expand Down
3 changes: 3 additions & 0 deletions pulsar-client-cpp/include/pulsar/c/authentication.h
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,9 @@ PULSAR_PUBLIC pulsar_authentication_t *pulsar_authentication_token_create(const
PULSAR_PUBLIC pulsar_authentication_t *pulsar_authentication_token_create_with_supplier(
token_supplier tokenSupplier, void *ctx);

PULSAR_PUBLIC pulsar_authentication_t *pulsar_authentication_basic_create(const char *username,
const char *password);

PULSAR_PUBLIC pulsar_authentication_t *pulsar_authentication_athenz_create(const char *authParamsString);

PULSAR_PUBLIC pulsar_authentication_t *pulsar_authentication_oauth2_create(const char *authParamsString);
Expand Down
7 changes: 7 additions & 0 deletions pulsar-client-cpp/lib/Authentication.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
#include "auth/AuthAthenz.h"
#include "auth/AuthToken.h"
#include "auth/AuthOauth2.h"
#include "auth/AuthBasic.h"
#include <lib/LogUtils.h>

#include <string>
Expand Down Expand Up @@ -129,6 +130,9 @@ AuthenticationPtr tryCreateBuiltinAuth(const std::string& pluginName, ParamMap&
} else if (boost::iequals(pluginName, OAUTH2_TOKEN_PLUGIN_NAME) ||
boost::iequals(pluginName, OAUTH2_TOKEN_JAVA_PLUGIN_NAME)) {
return AuthOauth2::create(paramMap);
} else if (boost::iequals(pluginName, BASIC_PLUGIN_NAME) ||
boost::iequals(pluginName, BASIC_JAVA_PLUGIN_NAME)) {
return AuthBasic::create(paramMap);
} else {
return AuthenticationPtr();
}
Expand All @@ -146,6 +150,9 @@ AuthenticationPtr tryCreateBuiltinAuth(const std::string& pluginName, const std:
} else if (boost::iequals(pluginName, OAUTH2_TOKEN_PLUGIN_NAME) ||
boost::iequals(pluginName, OAUTH2_TOKEN_JAVA_PLUGIN_NAME)) {
return AuthOauth2::create(authParamsString);
} else if (boost::iequals(pluginName, BASIC_PLUGIN_NAME) ||
boost::iequals(pluginName, BASIC_JAVA_PLUGIN_NAME)) {
return AuthBasic::create(authParamsString);
} else {
return AuthenticationPtr();
}
Expand Down
110 changes: 110 additions & 0 deletions pulsar-client-cpp/lib/auth/AuthBasic.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

#include "AuthBasic.h"

#include <stdexcept>
#include <boost/archive/iterators/base64_from_binary.hpp>
#include <boost/archive/iterators/transform_width.hpp>
#include <boost/property_tree/json_parser.hpp>
#include <boost/property_tree/ptree.hpp>
namespace ptree = boost::property_tree;

#include <sstream>
#include <functional>

namespace pulsar {

std::string base64_encode(const std::string& s) {
using namespace boost::archive::iterators;
using It = base64_from_binary<transform_width<std::string::const_iterator, 6, 8>>;
auto data = std::string(It(std::begin(s)), It(std::end(s)));
return data.append((3 - s.size() % 3) % 3, '=');
}

AuthDataBasic::AuthDataBasic(const std::string& username, const std::string& password) {
commandAuthToken_ = username + ":" + password;
httpAuthToken_ = base64_encode(commandAuthToken_);
}

AuthDataBasic::~AuthDataBasic() {}

bool AuthDataBasic::hasDataForHttp() { return true; }

std::string AuthDataBasic::getHttpHeaders() { return "Authorization: Basic " + httpAuthToken_; }

bool AuthDataBasic::hasDataFromCommand() { return true; }

std::string AuthDataBasic::getCommandData() { return commandAuthToken_; }

// AuthBasic

AuthBasic::AuthBasic(AuthenticationDataPtr &authDataBasic) { authDataBasic_ = authDataBasic; }

AuthBasic::~AuthBasic() = default;

AuthenticationPtr AuthBasic::create(const std::string& username, const std::string& password) {
AuthenticationDataPtr authDataBasic = AuthenticationDataPtr(new AuthDataBasic(username, password));
return AuthenticationPtr(new AuthBasic(authDataBasic));
}

ParamMap parseBasicAuthParamsString(const std::string& authParamsString) {
ParamMap params;
if (!authParamsString.empty()) {
ptree::ptree root;
std::stringstream stream;
stream << authParamsString;
try {
ptree::read_json(stream, root);
for (const auto& item : root) {
params[item.first] = item.second.get_value<std::string>();
}
} catch (ptree::json_parser_error& e) {
throw std::runtime_error(e.message());
}
}
return params;
}

AuthenticationPtr AuthBasic::create(const std::string& authParamsString) {
ParamMap paramMap = parseBasicAuthParamsString(authParamsString);
return create(paramMap);
}

AuthenticationPtr AuthBasic::create(ParamMap& params) {
auto usernameIt = params.find("username");
if (usernameIt == params.end()) {
throw std::runtime_error("Invalid username for basic provider");
}
auto passwordIt = params.find("password");
if (passwordIt == params.end()) {
throw std::runtime_error("Invalid password for basic provider");
}

return create(usernameIt->first, passwordIt->first);
}

const std::string AuthBasic::getAuthMethodName() const { return "basic"; }

Result AuthBasic::getAuthData(AuthenticationDataPtr& authDataBasic) {
authDataBasic = authDataBasic_;
return ResultOk;
}

} // namespace pulsar
46 changes: 46 additions & 0 deletions pulsar-client-cpp/lib/auth/AuthBasic.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

#pragma once

#include <pulsar/Authentication.h>
#include <string>
#include <boost/function.hpp>

namespace pulsar {

const std::string BASIC_PLUGIN_NAME = "basic";
const std::string BASIC_JAVA_PLUGIN_NAME = "org.apache.pulsar.client.impl.auth.AuthenticationBasic";

class AuthDataBasic : public AuthenticationDataProvider {
public:
AuthDataBasic(const std::string& username, const std::string& password);
~AuthDataBasic();

bool hasDataForHttp();
std::string getHttpHeaders();
bool hasDataFromCommand();
std::string getCommandData();

private:
std::string commandAuthToken_;
std::string httpAuthToken_;
};

} // namespace pulsar
4 changes: 4 additions & 0 deletions pulsar-client-cpp/pulsar-test-service-start.sh
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,10 @@ DATA_DIR=/tmp/pulsar-test-data
rm -rf $DATA_DIR
mkdir -p $DATA_DIR

# Set up basic authentication
cp $SRC_DIR/pulsar-client-cpp/test-conf/.htpasswd $DATA_DIR/.htpasswd
export PULSAR_EXTRA_OPTS=-Dpulsar.auth.basic.conf=$DATA_DIR/.htpasswd

# Copy TLS test certificates
mkdir -p $DATA_DIR/certs
cp $SRC_DIR/pulsar-broker/src/test/resources/authentication/tls/*.pem $DATA_DIR/certs
Expand Down
1 change: 1 addition & 0 deletions pulsar-client-cpp/test-conf/.htpasswd
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
admin:$apr1$FG4AO6aX$KGYPuMoLUou3i6vUkPUUf.
2 changes: 1 addition & 1 deletion pulsar-client-cpp/test-conf/standalone-ssl.conf
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ anonymousUserRole=anonymous
authenticationEnabled=true

# Authentication provider name list, which is comma separated list of class names
authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderTls,org.apache.pulsar.broker.authentication.AuthenticationProviderToken
authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderTls,org.apache.pulsar.broker.authentication.AuthenticationProviderToken,org.apache.pulsar.broker.authentication.AuthenticationProviderBasic

# Enforce authorization
authorizationEnabled=true
Expand Down
56 changes: 56 additions & 0 deletions pulsar-client-cpp/tests/AuthBasicTest.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

#include <pulsar/Authentication.h>

#include <gtest/gtest.h>
#include <pulsar/Client.h>

#include <string>

using namespace pulsar;

static const std::string serviceUrl = "pulsar://localhost:6650";
static const std::string serviceUrlHttp = "http://localhost:8080";

TEST(AuthPluginBasic, testBasic) {
ClientConfiguration config = ClientConfiguration();
AuthenticationPtr auth = pulsar::AuthBasic::create("admin", "123456");

ASSERT_TRUE(auth != NULL);
ASSERT_EQ(auth->getAuthMethodName(), "basic");

pulsar::AuthenticationDataPtr data;
ASSERT_EQ(auth->getAuthData(data), pulsar::ResultOk);
ASSERT_EQ(data->hasDataFromCommand(), true);
ASSERT_EQ(data->getCommandData(), "admin:123456");
ASSERT_EQ(data->hasDataForTls(), false);
ASSERT_EQ(data->hasDataForHttp(), true);
ASSERT_EQ(auth.use_count(), 1);

config.setAuth(auth);
Client client(serviceUrl, config);

std::string topicName = "persistent://private/auth/test-basic";
std::string subName = "subscription-name";

Producer producer;
Result result = client.createProducer(topicName, producer);
ASSERT_EQ(ResultOk, result);
}

0 comments on commit 9f556d3

Please sign in to comment.