Skip to content

Commit

Permalink
Merge pull request #5849 from graphcareful/license-time-as-unix-epoch
Browse files Browse the repository at this point in the history
security: Modify license timestamp to be unix epoch
  • Loading branch information
Rob Blafford committed Aug 5, 2022
2 parents 39c94a3 + 3cda1d5 commit 36a1945
Show file tree
Hide file tree
Showing 9 changed files with 69 additions and 113 deletions.
2 changes: 1 addition & 1 deletion src/go/rpk/pkg/api/admin/api_features.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ type LicenseProperties struct {
Version int `json:"format_version"`
Organization string `json:"org"`
Type string `json:"type"`
Expires int `json:"expires"`
Expires int64 `json:"expires"`
}

// GetFeatures returns information about the available features.
Expand Down
29 changes: 19 additions & 10 deletions src/go/rpk/pkg/cli/cmd/cluster/license/info.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import (
"encoding/json"
"fmt"
"os"
"time"

"github.com/redpanda-data/redpanda/src/go/rpk/pkg/api/admin"
"github.com/redpanda-data/redpanda/src/go/rpk/pkg/config"
Expand All @@ -22,7 +23,7 @@ func newInfoCommand(fs afero.Fs) *cobra.Command {
Organization: Organization the license was generated for.
Type: Type of license: free, enterprise, etc.
Expires: Number of days the license is valid until or -1 if is expired.
Expires: Expiration date of the license
Version: License schema version.
`,
Run: func(cmd *cobra.Command, args []string) {
Expand All @@ -37,16 +38,23 @@ func newInfoCommand(fs afero.Fs) *cobra.Command {
out.MaybeDie(err, "unable to retrieve license info: %v", err)

if !info.Loaded {
out.Die("this cluster is missing a license")
if format == "json" {
out.Die("{}")
} else {
out.Die("this cluster is missing a license")
}
}

if info.Properties != (admin.LicenseProperties{}) {
expired := info.Properties.Expires < 0
if format == "json" {
tm := time.Unix(info.Properties.Expires, 0).Format("Jan 2 2006")
props, err := json.MarshalIndent(struct {
admin.LicenseProperties
Expired bool `json:"license_expired,omitempty"`
}{info.Properties, expired}, "", " ")
Organization string
Type string
Expires string
Expired bool `json:"license_expired,omitempty"`
}{info.Properties.Organization, info.Properties.Type, tm, expired}, "", " ")
out.MaybeDie(err, "unable to print license information as json: %v", err)
fmt.Printf("%s\n", props)
} else {
Expand All @@ -66,16 +74,17 @@ func printLicenseInfo(p admin.LicenseProperties, expired bool) {
out.Section("LICENSE INFORMATION")
licenseFormat := `Organization: %v
Type: %v
Expires: %v days
Version: %v
Expires: %v
`
if expired {
licenseFormat += `License Expired: true
`
}
fmt.Printf(licenseFormat, p.Organization, p.Type, p.Expires, p.Version)
if p.Expires < 30 && p.Expires >= 0 {
tm := time.Unix(p.Expires, 0)
fmt.Printf(licenseFormat, p.Organization, p.Type, tm.Format("Jan 2 2006"))
diff := time.Until(tm)
daysLeft := int(diff.Hours() / 24)
if daysLeft < 30 && daysLeft >= 0 {
fmt.Fprintln(os.Stderr, "warning: your license will expire soon")
return
}
}
4 changes: 2 additions & 2 deletions src/v/redpanda/admin/api-doc/features.json
Original file line number Diff line number Diff line change
Expand Up @@ -142,8 +142,8 @@
"description": "type of license, currently only free_trial or enterprise"
},
"expires": {
"type": "int",
"description": "Number of days the license is valid until, -1 if is expired"
"type": "long",
"description": "Expiration date of the license in Unix epoch seconds"
}
}
},
Expand Down
2 changes: 1 addition & 1 deletion src/v/redpanda/admin_server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1706,7 +1706,7 @@ void admin_server::register_features_routes() {
lc.format_version = license->format_version;
lc.org = license->organization;
lc.type = security::license_type_to_string(license->type);
lc.expires = license->days_until_expires();
lc.expires = license->expiry.count();
res.license = lc;
}
co_return std::move(res);
Expand Down
51 changes: 14 additions & 37 deletions src/v/security/license.cc
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,6 @@
#include "json/validator.h"
#include "utils/base64.h"

#include <boost/date_time/gregorian/formatters.hpp>
#include <boost/date_time/gregorian/parsers.hpp>
#include <boost/filesystem.hpp>
#include <cryptopp/base64.h>
#include <cryptopp/rsa.h>
Expand Down Expand Up @@ -158,7 +156,7 @@ static const std::string license_data_validator_schema = R"(
"type": "number"
},
"expiry": {
"type": "string"
"type": "number"
}
},
"required": [
Expand All @@ -172,30 +170,16 @@ static const std::string license_data_validator_schema = R"(
)";

static void parse_data_section(license& lc, const json::Document& doc) {
auto parse_expiry = [&](auto& value) -> boost::gregorian::date {
auto expiry_str = value.GetString();
boost::gregorian::date expiry_date;
try {
expiry_date = boost::gregorian::from_simple_string(expiry_str);
} catch (const boost::bad_lexical_cast& ex) {
throw license_malformed_exception(
fmt::format("Bad cast: Expiry dateformat: {}", ex.what()));
}
if (expiry_date.is_not_a_date()) {
throw license_malformed_exception(
"Expiration date not a real calendar date");
}
const auto today = boost::gregorian::day_clock::universal_day();
if (expiry_date < today) {
throw license_invalid_exception("Expiry date behind todays date");
}
return expiry_date;
};
json::validator license_data_validator(license_data_validator_schema);
if (!doc.Accept(license_data_validator.schema_validator)) {
throw license_malformed_exception(
"License data section failed to match schema");
}
lc.expiry = std::chrono::seconds(
doc.FindMember("expiry")->value.GetInt64());
if (lc.is_expired()) {
throw license_invalid_exception("Expiry date behind todays date");
}
lc.format_version = doc.FindMember("version")->value.GetInt();
if (lc.format_version < 0) {
throw license_invalid_exception("Invalid format_version, is < 0");
Expand All @@ -205,7 +189,6 @@ static void parse_data_section(license& lc, const json::Document& doc) {
throw license_invalid_exception("Cannot have empty string for org");
}
lc.type = integer_to_license_type(doc.FindMember("type")->value.GetInt());
lc.expiry = parse_expiry(doc.FindMember("expiry")->value);
}

license make_license(const ss::sstring& raw_license) {
Expand All @@ -229,32 +212,26 @@ license make_license(const ss::sstring& raw_license) {
}

bool license::is_expired() const noexcept {
return expiry < boost::gregorian::day_clock::universal_day();
}

long license::days_until_expires() const noexcept {
return is_expired()
? -1
: (expiry - boost::gregorian::day_clock::universal_day()).days();
const auto now = std::chrono::duration_cast<std::chrono::seconds>(
std::chrono::system_clock::now().time_since_epoch());
return now > expiry;
}

void license::serde_read(iobuf_parser& in, const serde::header& h) {
using serde::read_nested;
format_version = read_nested<uint8_t>(in, h._bytes_left_limit);
type = read_nested<license_type>(in, h._bytes_left_limit);
organization = read_nested<ss::sstring>(in, h._bytes_left_limit);
expiry = boost::gregorian::from_simple_string(
read_nested<ss::sstring>(in, h._bytes_left_limit));
expiry = std::chrono::seconds(
read_nested<int64_t>(in, h._bytes_left_limit));
}

void license::serde_write(iobuf& out) {
using serde::write;
write(out, format_version);
write(out, type);
write(out, organization);
write(
out,
ss::sstring(boost::gregorian::to_iso_extended_string_type<char>(expiry)));
write(out, static_cast<int64_t>(expiry.count()));
}

} // namespace security
Expand All @@ -268,11 +245,11 @@ fmt::formatter<security::license, char, void>::format<
fmt::basic_format_context<fmt::appender, char>& ctx) const {
return format_to(
ctx.out(),
"[Version: {0}, Organization: {1}, Type: {2} Expiry(days): {3}]",
"[Version: {0}, Organization: {1}, Type: {2} Expiry(epoch): {3}]",
r.format_version,
r.organization,
license_type_to_string(r.type),
r.days_until_expires());
r.expiry.count());
}

} // namespace fmt
6 changes: 3 additions & 3 deletions src/v/security/license.h
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ struct license : serde::envelope<license, serde::version<0>> {
uint8_t format_version;
license_type type;
ss::sstring organization;
boost::gregorian::date expiry;
std::chrono::seconds expiry;

/// Explicit serde:: implementations because boost::gregorian is not
/// trivally serializable/deserializable
Expand All @@ -77,8 +77,8 @@ struct license : serde::envelope<license, serde::version<0>> {
/// true if todays date is greater then \ref expiry
bool is_expired() const noexcept;

/// returns -1 in the case the license has already expired
long days_until_expires() const noexcept;
/// Seconds since epoch until license expiration
std::chrono::seconds expires() const noexcept;

private:
friend struct fmt::formatter<license>;
Expand Down
47 changes: 20 additions & 27 deletions src/v/security/tests/license_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,9 @@ BOOST_AUTO_TEST_CASE(test_license_invalid_signature) {
/// This license has been generated with a non matching signature, even
/// though the contents section is valid
static const auto license_contents_bad_signature
= "eyJ2ZXJzaW9uIjogMCwgIm9yZyI6ICJyZWRwYW5kYS10ZXN0aW5nIiwgInR5cGUiOiAxLC"
"AiZXhwaXJ5IjogIjIwMjItNy00In0=."
"QOPTqwnMwV0hc4ZwsPBQkI9LPmwSgiHGWWxVGutk+"
"THrXDtm2UTFxhFpGEYgdmBNCNLKKiBNfyMSsohBAMCr6U5k3211d7+X1++"
"6ni3ykinJBhuLYE9+fNnHHxF/"
"GnkOGzFwklOUKXR7CuSEMrmUt6cDLkoyjTAtE6ygi7T8hOpbEOf5B9IK72IgSsFw7phtEl"
"uy9gFl+XkQZy5cx8roHd/G6PRgirN/"
"zDeyC66vIjIU8ZNV4Ly+69asRRakvh6lnLbGIpfmWwUJKi9DX4DEGaw/"
"WGYKJB5jzywOcoNdO/t4AT8UeCArICiPmrqCvISvJJk80OKiSEU3ChLNHJPRsQ==";
= "eyJ2ZXJzaW9uIjogMCwgIm9yZyI6ICJyZWRwYW5kYS1jbG91ZCIsICJ0eXBlIjogMSwgIm"
"V4cGlyeSI6IDE2NjA0OTg1MzZ9.dfadf/dfadfa+kkk/I/kk/"
"934349asfkdw==";
BOOST_CHECK_THROW(
make_license(license_contents_bad_signature),
license_verifcation_exception);
Expand All @@ -37,13 +31,14 @@ BOOST_AUTO_TEST_CASE(test_license_malformed_content) {
/// This license has been generated without the 'expiry' parameter, making
/// it malformed
static const auto license_contents_malformed_content
= "eyJoYWkiOiAiMTIzIn0=."
"KM7DNlb3Ja49xImVi6FnwewXXp2Skt72q7RV8xcBCQtlg7frEzTbQmu6eKq2scSU2zqOX5"
"FBqJ1ZEZ6RaaSiEqVGrsvHfR8bh4qSSzWUP8ny+"
"wcpei8zBfRUR2ulZv9rib3FPKDlNHC3Smtsyosim+"
"i2O7A3ARPKHFFBFtKufKTihHPY87JxY8ytAXWNlCfaisaPst9XQtUmy4iJAx5QrJpC4Y03"
"u9XOC0/cbwKzotMGP8TojU2V5/zlxMde/VYWI3ic5Jhp4x5rHTDNmV2eaUaDU8h3W55D/"
"UJ/feVi5ba2wfBFto/0uZ1M7NvevmfmQ3UC3z6bJnmkdwqZ2TdpWQ==";
= "eyJ2ZXJzaW9uIjogMCwgIm9yZyI6ICJyZWRwYW5kYS1jbG91ZCIsICJ0eXBlIjogMX0=."
"F2EHtQftac9+O3ucUijiJ6ta5nmoonEoZhr86FCA+"
"4hAIQnetBcO1h7yD4OUHy7t9dS0hTz2BJU981G8i0Ud+v0+"
"GRBII3VSZ1gL3W98QRGE1KiOjR11G3+8YQhSrFRJMHhXyYPEHiFKuYDCoIaozd2IhGYi/"
"Gcnq/yWZRRDTcnhPOBQivkX5lQpTPorM+xO5ER4wrAROm2jp2lB/"
"GDrco1f5iU9M3anIZo1F3rq4V0rnH/FJvwQW/"
"7gwO+Ut06O3lWJoUZOTuwueyxopACRmWOm/"
"DOYPZDkc8Xoui69EKVqRc4UOujbfOBYzhGq7wNlchJ0QOKUz9Bi/ZPoedOFAw==";
BOOST_CHECK_THROW(
make_license(license_contents_malformed_content),
license_malformed_exception);
Expand All @@ -53,14 +48,14 @@ BOOST_AUTO_TEST_CASE(test_license_invalid_content) {
/// This license was generated with an expiration date set to a date in the
/// past, making it invalid
static const auto license_contents_invalid_content
= "eyJ2ZXJzaW9uIjogMCwgIm9yZyI6ICJyZWRwYW5kYS10ZXN0aW5nIiwgInR5cGUiOiAxLC"
"AiZXhwaXJ5IjogIjE5OTktNy00In0=.RKLx88ZUtzAQofO3F8azuUn8k9q+"
"tS37JvsjwZs7YHluupuAQXpQJk2qLVWlJeaMvjhaQTXNl6j7JEoKbUmJESnOjh5ghre64x"
"YPF5jLkN+S1N0eoVp0eR7w13vo3RVwfkKWLZKM7JTXdXJiXHqvnXrtjXpCR5T+"
"P39KJFDeOTcwY6ojcBJVcYidpvExfNx9S/"
"N0Lw4txozdywaYT3W4xABr8k0KlXmf8Oag77qW3kAcKHmjin6R64GTrcDSx/"
"SQY18KjPw9J9s2gZRXIHo6U0Jmsv6lNbDAkPtUAN+AQRTBN4ayQEz40yqxO279vz3U4UO/"
"4SbXfVdZ524rEmTrMQ==";
= "eyJ2ZXJzaW9uIjogMCwgIm9yZyI6ICJyZWRwYW5kYS1jbG91ZCIsICJ0eXBlIjogMSwgIm"
"V4cGlyeSI6IDE2NTg3NzA0Nzh9.m2fIYroOtmIEaJILcGUyDPPDLbJStO+"
"20GnMbA9Gg9QHVMLihj4dgL7k4x+kKqRMSCVe5tkhkNS/"
"2pzcIranGbWs7AlShBFsXKKA8rTYW1xhvVYw71gkPHkyCOXb3++tcmU5W0MSQM9r4/"
"XTNHz7DfM4bvWsw9IM4tWeIC3U+SoiJ+ARXag7wxETo7JGgS4+AB7WIj3u9whVA1+"
"6p9w0/"
"LgRUeIoVru6frBAVHUCGl6x2npqoLTzMRT2d3YnFnI8ilBeQllq7bTAcNkQwXwKigfcBe2"
"WSj/n77O/GNTlIhBVBtbBM2EcbZQMAhrSnTJJg5kcQMMg9oVjzg278cO+hw==";
BOOST_CHECK_THROW(
make_license(license_contents_invalid_content),
license_invalid_exception);
Expand All @@ -81,8 +76,6 @@ BOOST_AUTO_TEST_CASE(test_license_valid_content) {
BOOST_CHECK_EQUAL(license.format_version, 0);
BOOST_CHECK_EQUAL(license.type, license_type::enterprise);
BOOST_CHECK_EQUAL(license.organization, "redpanda-testing");
BOOST_CHECK(
license.expiry == boost::gregorian::from_simple_string("2122-06-06"));
BOOST_CHECK(!license.is_expired());
BOOST_CHECK_EQUAL(license.expiry.count(), 4813252273);
}
} // namespace security
19 changes: 3 additions & 16 deletions tests/rptest/tests/cluster_features_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -158,8 +158,8 @@ def test_license_upload_and_query(self):
self.logger.info(
"Skipping test, REDPANDA_SAMPLE_LICENSE env var not found")
return
license_contents = {
'expires': datetime.date(2122, 6, 6),
expected_license_contents = {
'expires': 4813252273,
'format_version': 0,
'org': 'redpanda-testing',
'type': 'enterprise'
Expand All @@ -172,20 +172,7 @@ def test_license_upload_and_query(self):
resp = self.admin.get_license()
assert resp['loaded'] is True
assert resp['license'] is not None

def is_equal_to_license_properties(license_contents,
license_properties):
"""Compares the values within first parameters map to a response
from the redpanda admin server"""
days_left = (license_contents['expires'] -
datetime.date.today()).days
return license_properties['format_version'] == license_contents['format_version'] and \
license_properties['org'] == license_contents['org'] and \
license_properties['type'] == license_contents['type'] and \
license_properties['expires'] == days_left

assert is_equal_to_license_properties(license_contents,
resp['license']) is True
assert expected_license_contents == resp['license'], resp['license']


class FeaturesMultiNodeUpgradeTest(FeaturesTestBase):
Expand Down
22 changes: 6 additions & 16 deletions tests/rptest/tests/rpk_cluster_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -206,29 +206,19 @@ def test_upload_and_query_cluster_license_rpk(self):
output = self._rpk.license_set(tf.name)
assert "Successfully uploaded license" in output

def get_license():
output = self._rpk.license_info()
resp = json.loads(output)
if resp['org'] == "redpanda-testing":
return True

return False

wait_until(get_license,
wait_until(lambda: self._rpk.license_info() != "{}",
timeout_sec=10,
backoff_sec=1,
retry_on_exc=True,
err_msg="unable to retrieve license information")

expected_license = {
'expires':
(datetime.date(2122, 6, 6) - datetime.date.today()).days,
'format_version': 0,
'org': 'redpanda-testing',
'type': 'enterprise'
'Expires': "Jul 11 2122",
'Organization': 'redpanda-testing',
'Type': 'enterprise'
}
output = self._rpk.license_info()
assert expected_license == json.loads(output)
result = json.loads(self._rpk.license_info())
assert expected_license == result, result

@cluster(num_nodes=3)
def test_upload_cluster_license_rpk(self):
Expand Down

0 comments on commit 36a1945

Please sign in to comment.