Two classes exist for filtering.
Collections::Filter
Collections::VB::Filter
The idea is that a DCP producer will establish a Collections::Filter
that lives for the lifetime of the DCP producer.
As the DCP producer creates streams, a Collections::VB::Filter is
assigned to the stream which contains the real set of collections to
filter (and also the actual "filter" function).
Change-Id: I2f35b1698ce977116486a2e6940437eee25faef1
Reviewed-on: http://review.couchbase.org/78137
Reviewed-by: Dave Rigby <daver@couchbase.com>
Tested-by: Build Bot <build@couchbase.com>
SET(CONFIG_SOURCE src/configuration.cc
${CMAKE_CURRENT_BINARY_DIR}/src/generated_configuration.cc)
-SET(COLLECTIONS_SOURCE src/collections/manager.cc
+SET(COLLECTIONS_SOURCE src/collections/filter.cc
+ src/collections/manager.cc
src/collections/manifest.cc
+ src/collections/vbucket_filter.cc
src/collections/vbucket_manifest.cc
src/collections/vbucket_manifest_entry.cc)
tests/module_tests/checkpoint_test.cc
tests/module_tests/collections/collection_dockey_test.cc
tests/module_tests/collections/evp_store_collections_test.cc
+ tests/module_tests/collections/filter_test.cc
tests/module_tests/collections/manifest_test.cc
tests/module_tests/collections/vbucket_manifest_test.cc
tests/module_tests/collections/vbucket_manifest_entry_test.cc
--- /dev/null
+/* -*- Mode: C++; tab-width: 4; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+/*
+ * Copyright 2017 Couchbase, Inc
+ *
+ * Licensed 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 "collections/filter.h"
+#include "collections/collections_dockey.h"
+#include "collections/collections_types.h"
+
+#include <JSON_checker.h>
+#include <cJSON.h>
+#include <cJSON_utils.h>
+#include <platform/make_unique.h>
+
+#include <iostream>
+
+/**
+ * Create a filter from a JSON object with the following format:
+ *
+ * {"collection":["collection1", "collection2"]}
+ *
+ * Each element of the "collection" array must be collections known to the
+ * specified Manifest.
+ *
+ * @param jsonFilter a buffer containing an optional JSON object to initialise
+ * from
+ * @param manifest the Manifest (bucket manifest) to check the filter is valid
+ * @throws invalid_argument for input errors (with detailed message)
+ */
+Collections::Filter::Filter(boost::optional<const std::string&> jsonFilter,
+ const Manifest& manifest)
+ : defaultAllowed(false),
+ passthrough(false),
+ systemEventsAllowed(true),
+ separator(manifest.getSeparator()) {
+ // If no filter is specified at all, then create a default collection only
+ // filter. This filter means the default collection is streamed
+ if (!jsonFilter.is_initialized()) {
+ // If $default is a collection, then let's filter it
+ if (manifest.doesDefaultCollectionExist()) {
+ defaultAllowed = true;
+
+ // This filter is for a 'legacy' default only user, so they should
+ // never see system events.
+ systemEventsAllowed = false;
+ return;
+ } else {
+ throw std::invalid_argument(
+ "Filter::Filter default collection does not exist");
+ }
+ }
+
+ auto json = jsonFilter.get();
+
+ // An empty filter is specified so all documents are available
+ if (json.empty()) {
+ passthrough = true;
+ defaultAllowed = true;
+ return;
+ }
+
+ if (!checkUTF8JSON(reinterpret_cast<const unsigned char*>(json.data()),
+ json.size())) {
+ throw std::invalid_argument(
+ "Filter::Filter input not valid jsonFilter:" +
+ jsonFilter.get());
+ }
+
+ unique_cJSON_ptr cjson(cJSON_Parse(json.c_str()));
+ if (!cjson) {
+ throw std::invalid_argument(
+ "Filter::Filter cJSON cannot parse jsonFilter:" +
+ jsonFilter.get());
+ }
+
+ auto jsonCollections = cJSON_GetObjectItem(cjson.get(), "collections");
+ if (!jsonCollections || jsonCollections->type != cJSON_Array) {
+ throw std::invalid_argument(
+ "Filter::Filter cannot find collections:" +
+ (!jsonCollections ? "nullptr"
+ : std::to_string(jsonCollections->type)) +
+ ", jsonFilter:" + jsonFilter.get());
+ } else {
+ for (int ii = 0; ii < cJSON_GetArraySize(jsonCollections); ii++) {
+ auto collection = cJSON_GetArrayItem(jsonCollections, ii);
+ if (!collection || collection->type != cJSON_String) {
+ throw std::invalid_argument(
+ "Filter::Filter cannot find "
+ "valid collection for index:" +
+ std::to_string(ii) + ", collection:" +
+ (!collection ? "nullptr"
+ : std::to_string(collection->type)) +
+ ", jsonFilter:" + jsonFilter.get());
+ } else {
+ // Can throw..
+ addCollection(collection->valuestring, manifest);
+ }
+ }
+ }
+}
+
+void Collections::Filter::addCollection(const char* collection,
+ const Manifest& manifest) {
+ // Is this the default collection?
+ if (DefaultCollectionIdentifier == collection) {
+ if (manifest.doesDefaultCollectionExist()) {
+ defaultAllowed = true;
+ } else {
+ throw std::invalid_argument(
+ "Filter::Filter: $default is not a known collection");
+ }
+ } else {
+ if (manifest.find({collection}) != manifest.end()) {
+ filter.push_back(collection);
+ } else {
+ throw std::invalid_argument("Filter::Filter: collection:" +
+ std::string(collection) +
+ " is not a known collection");
+ }
+ }
+}
+
+void Collections::Filter::dump() const {
+ std::cerr << *this << std::endl;
+}
+
+std::ostream& Collections::operator<<(std::ostream& os,
+ const Collections::Filter& filter) {
+ os << "Collections::Filter"
+ << ": passthrough:" << filter.passthrough
+ << ", defaultAllowed:" << filter.defaultAllowed
+ << ", systemEventsAllowed:" << filter.systemEventsAllowed
+ << ", separator:" << filter.separator
+ << ", filter.size:" << filter.filter.size() << std::endl;
+ for (const auto& entry : filter.filter) {
+ os << entry << std::endl;
+ }
+ return os;
+}
--- /dev/null
+/* -*- Mode: C++; tab-width: 4; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+/*
+ * Copyright 2017 Couchbase, Inc
+ *
+ * Licensed 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 "collections/manifest.h"
+
+#include <boost/optional/optional.hpp>
+#include <memcached/dockey.h>
+#include <platform/sized_buffer.h>
+
+#include <string>
+#include <unordered_map>
+
+namespace Collections {
+
+/**
+ * Collections::Filter stores the JSON filter which DCP_OPEN_PRODUCER can
+ * specify.
+ *
+ * A Collections::Filter is optional in that the client can omit a filter
+ * in which case the filter is a pass-through (isPassthrough():true).
+ *
+ * This object is used to create Collections::VB::Filter objects when VB streams
+ * are requested. The Collections::VB::Filter object is used to make the final
+ * decision if data should be streamed or dropped.
+ */
+class Filter {
+public:
+ /**
+ * Construct a Collections::Filter using an optional JSON document
+ * and the bucket's current Manifest.
+ *
+ * The optional JSON document allows a client to filter a chosen set of
+ * collections or just the default collection.
+ *
+ * if jsonFilter is defined and empty - then we create a passthrough.
+ * if jsonFilter is defined and !empty - then we filter as requested.
+ * if jsonFilter is not defined (maybe a legacy client who doesn't
+ * understand collections) then only documents with
+ * DocNamespace::DefaultCollection are allowed.
+ *
+ * @throws invalid_argument if the JSON is invalid or contains unknown
+ * collections.
+ */
+ Filter(boost::optional<const std::string&> jsonFilter,
+ const Manifest& manifest);
+
+ /**
+ * Get the list of collections the filter knows about. Can be empty
+ * @returns std::vector of std::string, maybe empty for a passthrough filter
+ */
+ const std::vector<std::string>& getFilter() const {
+ return filter;
+ }
+
+ /**
+ * @returns if the filter configured so that it allows everything through?
+ */
+ bool isPassthrough() const {
+ return passthrough;
+ }
+
+ /**
+ * @returns if the filter contains the default collection
+ */
+ bool allowDefaultCollection() const {
+ return defaultAllowed;
+ }
+
+ /**
+ * @returns if the filter should allow system events
+ */
+ bool allowSystemEvents() const {
+ return systemEventsAllowed;
+ }
+
+ /**
+ * Dump this to std::cerr
+ */
+ void dump() const;
+
+private:
+ /**
+ * Private helper to examine the given collection name against the manifest
+ * and add to internal container or throw an exception
+ */
+ void addCollection(const char* collection, const Manifest& manifest);
+
+ std::vector<std::string> filter;
+ bool defaultAllowed;
+ bool passthrough;
+ bool systemEventsAllowed;
+
+ /**
+ * const copy of the collections separator
+ * @todo MB-24507 - filters and the separator changing
+ */
+ const std::string separator;
+
+ friend std::ostream& operator<<(std::ostream& os, const Filter& filter);
+};
+
+std::ostream& operator<<(std::ostream& os, const Filter& filter);
+
+} // end namespace Collections
namespace Collections {
-Manifest::Manifest() : revision(0), separator(DefaultSeparator) {
+Manifest::Manifest()
+ : revision(0), defaultCollectionExists(true), separator(DefaultSeparator) {
collections.push_back(DefaultCollectionIdentifier.data());
}
-Manifest::Manifest(const std::string& json) {
+Manifest::Manifest(const std::string& json)
+ : revision(-1), defaultCollectionExists(false) {
if (!checkUTF8JSON(reinterpret_cast<const unsigned char*>(json.data()),
json.size())) {
throw std::invalid_argument("Manifest::Manifest input not valid json");
(!collection ? " nullptr"
: std::to_string(collection->type)));
} else if (validCollection(collection->valuestring)) {
+ if (std::strncmp(collection->valuestring,
+ DefaultCollectionIdentifier.data(),
+ DefaultCollectionIdentifier.size()) == 0) {
+ defaultCollectionExists = true;
+ }
collections.push_back(collection->valuestring);
} else {
throw std::invalid_argument(
return separator;
}
+ bool doesDefaultCollectionExist() const {
+ return defaultCollectionExists;
+ }
+
std::vector<std::string>::const_iterator begin() const {
return collections.begin();
}
static bool validCollection(const char* collection);
int revision;
+ bool defaultCollectionExists;
std::string separator;
std::vector<std::string> collections;
};
--- /dev/null
+/* -*- Mode: C++; tab-width: 4; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+/*
+ * Copyright 2017 Couchbase, Inc
+ *
+ * Licensed 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 "collections/vbucket_filter.h"
+#include "collections/collections_dockey.h"
+#include "collections/filter.h"
+#include "collections/vbucket_manifest.h"
+#include "dcp/response.h"
+#include "statwriter.h"
+
+#include <platform/checked_snprintf.h>
+#include <platform/make_unique.h>
+
+Collections::VB::Filter::Filter(const Collections::Filter& filter,
+ const Collections::VB::Manifest& manifest)
+ : defaultAllowed(false),
+ passthrough(filter.isPassthrough()),
+ systemEventsAllowed(filter.allowSystemEvents()) {
+ // Don't build a filter if all documents are allowed
+ if (passthrough) {
+ defaultAllowed = true;
+ return;
+ }
+
+ // Lock for reading and create a VB filter
+ auto rh = manifest.lock();
+ separator = rh.getSeparator();
+ if (filter.allowDefaultCollection()) {
+ if (rh.doesDefaultCollectionExist()) {
+ defaultAllowed = true;
+ } else {
+ // The VB::Manifest no longer has $default so don't filter it
+ LOG(EXTENSION_LOG_NOTICE,
+ "VB::Filter::Filter: dropping $default as it's not in the "
+ "VB::Manifest");
+ }
+ }
+
+ for (const auto& c : filter.getFilter()) {
+ if (rh.doesCollectionExist({c.data(), c.size()})) {
+ auto m = std::make_unique<std::string>(c);
+ cb::const_char_buffer b{m->data(), m->size()};
+ this->filter.emplace(b, std::move(m));
+ } else {
+ // The VB::Manifest no longer has the collection so we won't filter
+ // it
+ LOG(EXTENSION_LOG_NOTICE,
+ "VB::Filter::Filter: dropping collection:%s as it's not in the "
+ "VB::Manifest",
+ c.c_str());
+ }
+ }
+}
+
+bool Collections::VB::Filter::allow(::DocKey key) const {
+ // passthrough, everything is allowed.
+ if (passthrough) {
+ return true;
+ }
+
+ // The presence of $default is a simple check against defaultAllowed
+ if (key.getDocNamespace() == DocNamespace::DefaultCollection &&
+ defaultAllowed) {
+ return true;
+ } else if (key.getDocNamespace() == DocNamespace::Collections &&
+ !filter.empty()) {
+ // Collections require a look up in the filter
+ const auto cKey = Collections::DocKey::make(key, separator);
+ return filter.count({reinterpret_cast<const char*>(cKey.data()),
+ cKey.getCollectionLen()}) > 0;
+ } else if (key.getDocNamespace() == DocNamespace::System) {
+ // ::allow should only be called for the Default or Collection namespace
+ throw std::invalid_argument(
+ "Collections::VB::Filter::allow namespace system invalid:" +
+ std::to_string(int(key.getDocNamespace())));
+ }
+ return false;
+}
+
+bool Collections::VB::Filter::remove(cb::const_char_buffer collection) {
+ if (passthrough) {
+ // passthrough can never be empty, so return false
+ return false;
+ }
+
+ if (collection == DefaultCollectionIdentifier) {
+ defaultAllowed = false;
+ } else {
+ filter.erase(collection);
+ }
+
+ // If the map is empty and the defaultCollection isn't present, we're empty
+ return filter.empty() && !defaultAllowed;
+}
+
+bool Collections::VB::Filter::allowSystemEvent(
+ SystemEventMessage* response) const {
+ switch (response->getSystemEvent()) {
+ case SystemEvent::CreateCollection:
+ case SystemEvent::BeginDeleteCollection: {
+ if ((response->getKey() == DefaultCollectionIdentifier &&
+ defaultAllowed) ||
+ passthrough) {
+ return true;
+ } else {
+ // These events are sent only if they relate to a collection in the
+ // filter
+ return filter.count(response->getKey()) > 0;
+ }
+ }
+ case SystemEvent::CollectionsSeparatorChanged:
+ // The separator changed event is sent if system events are allowed
+ return systemEventsAllowed;
+ case SystemEvent::DeleteCollectionHard:
+ case SystemEvent::DeleteCollectionSoft:
+ break;
+ }
+ throw std::invalid_argument(
+ "SystemEventReplicate::filter event:" +
+ std::to_string(int(response->getSystemEvent())) +
+ " should not be present in SystemEventMessage");
+}
+
+void Collections::VB::Filter::addStats(ADD_STAT add_stat,
+ const void* c,
+ const std::string& prefix,
+ uint16_t vb) const {
+ try {
+ const int bsize = 1024;
+ char buffer[bsize];
+ checked_snprintf(
+ buffer, bsize, "%s:filter_%d_passthrough", prefix.c_str(), vb);
+ add_casted_stat(buffer, passthrough, add_stat, c);
+
+ checked_snprintf(buffer,
+ bsize,
+ "%s:filter_%d_default_allowed",
+ prefix.c_str(),
+ vb);
+ add_casted_stat(buffer, defaultAllowed, add_stat, c);
+
+ checked_snprintf(
+ buffer, bsize, "%s:filter_%d_size", prefix.c_str(), vb);
+ add_casted_stat(buffer, filter.size(), add_stat, c);
+ } catch (std::exception& error) {
+ LOG(EXTENSION_LOG_WARNING,
+ "Collections::VB::Filter::addStats: %s:vb:%" PRIu16
+ " exception.what:%s",
+ prefix.c_str(),
+ vb,
+ error.what());
+ }
+}
+
+void Collections::VB::Filter::dump() const {
+ std::cerr << *this << std::endl;
+}
+
+std::ostream& Collections::VB::operator<<(
+ std::ostream& os, const Collections::VB::Filter& filter) {
+ os << "VBucket::Filter"
+ << ": defaultAllowed:" << filter.defaultAllowed
+ << ", passthrough:" << filter.passthrough
+ << ", systemEventsAllowed:" << filter.systemEventsAllowed;
+
+ if (filter.separator.empty()) {
+ os << ", separator empty";
+ } else {
+ os << ", separator:" << filter.separator;
+ }
+
+ os << ", filter.size:" << filter.filter.size() << std::endl;
+ for (auto& m : filter.filter) {
+ os << *m.second << std::endl;
+ }
+ return os;
+}
\ No newline at end of file
--- /dev/null
+/* -*- Mode: C++; tab-width: 4; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+/*
+ * Copyright 2017 Couchbase, Inc
+ *
+ * Licensed 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 <memcached/dockey.h>
+#include <memcached/engine_common.h>
+#include <platform/sized_buffer.h>
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+class SystemEventMessage;
+
+namespace Collections {
+
+class Filter;
+
+namespace VB {
+
+class Manifest;
+
+/**
+ * The VB filter is used to decide if keys on a DCP stream should be sent
+ * or dropped.
+ *
+ * A filter is built from the Collections::Filter that was established when
+ * the producer was opened. During the time the producer was opened and a
+ * stream is requested, filtered collections may have been deleted, so the
+ * ::VB::Filter becomes the intersection of the producer's filter and the
+ * open collections within the manifest.
+ *
+ * Note: There is no locking on a VB::Filter as at the moment it is constructed
+ * and then is not mutable.
+ */
+class Filter {
+public:
+ /**
+ * Construct a Collections::VB::Filter using the producer's
+ * Collections::Filter and the vbucket's collections manifest.
+ *
+ * If the producer's filter is configured to filter collections then the
+ * resulting object will filter the intersection filter:manifest
+ * collections. The constructor will log when it finds it must drop a
+ * collection
+ *
+ * If the producer's filter is effectively a passthrough
+ * (allowAllCollections returns true) then so will the resulting VB filter.
+ *
+ * @param filter The producer's filter that the client configured.
+ * @param manifest The vbucket's collection manifest.
+ */
+ Filter(const ::Collections::Filter& filter,
+ const ::Collections::VB::Manifest& manifest);
+
+ /**
+ * @returns if the filter allows key based on the filter contents
+ */
+ bool allow(::DocKey key) const;
+
+ /**
+ * Attempt to remove the collection from the filter, no-op if the filter
+ * does include the collection.
+ *
+ * @param collection a collection name to remove.
+ * @returns true if the filter is empty
+ */
+ bool remove(cb::const_char_buffer collection);
+
+ /**
+ * Does the filter allow the system event? I.e. a "meat,dairy" filter
+ * shouldn't allow delete events for the "fruit" collection.
+ *
+ * @param response a SystemEventMessage to check
+ * @param return true if the filter says this event should be allowed
+ */
+ bool allowSystemEvent(SystemEventMessage* response) const;
+
+ /**
+ * Add statistics for this filter, currently just depicts the object's state
+ */
+ void addStats(ADD_STAT add_stat,
+ const void* c,
+ const std::string& prefix,
+ uint16_t vb) const;
+
+ /**
+ * Dump this to std::cerr
+ */
+ void dump() const;
+
+private:
+ using Container = ::std::unordered_map<cb::const_char_buffer,
+ std::unique_ptr<std::string>>;
+ Container filter;
+ bool defaultAllowed;
+ bool passthrough;
+ bool systemEventsAllowed;
+ std::string separator;
+
+ friend std::ostream& operator<<(std::ostream& os, const Filter& filter);
+};
+
+std::ostream& operator<<(std::ostream& os, const Filter& filter);
+
+} // end namespace VB
+} // end namespace Collections
}
/**
- * @returns the current separator
+ * @returns a copy of the current separator
*/
std::string getSeparator() const {
return manifest.getSeparator();
}
+ /**
+ * @returns true/false if $default exists
+ */
+ bool doesDefaultCollectionExist() const {
+ return manifest.doesDefaultCollectionExist();
+ }
+
+ /**
+ * @returns true/false if the collection exists
+ */
+ bool doesCollectionExist(cb::const_char_buffer collection) const {
+ return manifest.doesCollectionExist(collection);
+ }
+
private:
std::unique_lock<cb::ReaderLock> readLock;
const Manifest& manifest;
return separator;
}
+
+ /**
+ * @returns true/false if $default exists
+ */
+ bool doesDefaultCollectionExist() const {
+ return defaultCollectionExists;
+ }
+
+ /**
+ * @returns true/false if the collection exists
+ */
+ bool doesCollectionExist(cb::const_char_buffer collection) const {
+ return map.count(collection) != 0;
+ }
+
protected:
/**
* Add a collection entry to the manifest specifing the revision that it was
--- /dev/null
+/* -*- Mode: C++; tab-width: 4; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+/*
+ * Copyright 2017 Couchbase, Inc
+ *
+ * Licensed 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 "collections/filter.h"
+#include "collections/vbucket_filter.h"
+#include "collections/vbucket_manifest.h"
+#include "ep_vb.h"
+#include "failover-table.h"
+
+#include <gtest/gtest.h>
+
+#include <limits>
+
+class CollectionsFilterTest : public ::testing::Test {
+public:
+ /// Dummy callback to replace the flusher callback so we can create VBuckets
+ class DummyCB : public Callback<uint16_t> {
+ public:
+ DummyCB() {
+ }
+
+ void callback(uint16_t& dummy) {
+ }
+ };
+
+ CollectionsFilterTest()
+ : vb(0,
+ vbucket_state_active,
+ global_stats,
+ checkpoint_config,
+ /*kvshard*/ nullptr,
+ /*lastSeqno*/ 0,
+ /*lastSnapStart*/ 0,
+ /*lastSnapEnd*/ 0,
+ /*table*/ nullptr,
+ std::make_shared<DummyCB>(),
+ /*newSeqnoCb*/ nullptr,
+ config,
+ VALUE_ONLY) {
+ }
+
+ EPStats global_stats;
+ CheckpointConfig checkpoint_config;
+ Configuration config;
+ EPVBucket vb;
+};
+
+/**
+ * Test invalid inputs to the filter.
+ */
+TEST_F(CollectionsFilterTest, junk_in) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"::",)"
+ R"("collections":["$default", "vegetable"]})");
+
+ std::vector<std::string> inputs = {"{}",
+ R"({"collections":1})",
+ R"({"collections:"this"})",
+ R"({"collections:{"a":1})",
+ R"({"collection:["a"])",
+ R"({"collections:[a])"};
+
+ for (const auto& s : inputs) {
+ boost::optional<const std::string&> json = s;
+ EXPECT_THROW(std::make_unique<Collections::Filter>(json, m),
+ std::invalid_argument)
+ << "Failed for " << s;
+ }
+}
+
+/**
+ * Test valid inputs to the filter.
+ */
+TEST_F(CollectionsFilterTest, validation1) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"::",)"
+ R"("collections":["$default", "vegetable", "fruit", "meat", "dairy"]})");
+
+ std::vector<std::string> inputs = {R"({"collections":["$default"]})",
+ R"({"collections":["vegetable"]})",
+ R"({"collections":["fruit", "meat"]})"};
+
+ for (const auto& s : inputs) {
+ boost::optional<const std::string&> json = s;
+
+ EXPECT_NO_THROW(std::make_unique<Collections::Filter>(json, m))
+ << "Exception thrown with input " << s;
+ }
+}
+
+/**
+ * Test valid inputs to the filter, but they are not known collections, so
+ * should trigger an exception.
+ */
+TEST_F(CollectionsFilterTest, validation2) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"::",)"
+ R"("collections":["$default", "vegetable", "fruit", "meat", "dairy"]})");
+
+ std::vector<std::string> inputs = {R"({"collections":["cheese"]})",
+ R"({"collections":["fruit","beer"]})",
+ R"({"collections":["$dufault"]})"};
+
+ for (const auto& s : inputs) {
+ boost::optional<const std::string&> json = s;
+ EXPECT_THROW(std::make_unique<Collections::Filter>(json, m),
+ std::invalid_argument);
+ }
+}
+
+/**
+ * Test that we cannot create default collection filter when no default
+ * collection exists
+ */
+TEST_F(CollectionsFilterTest, validation_no_default) {
+ // m does not include $default
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"::",)"
+ R"("collections":["vegetable", "fruit", "meat", "dairy"]})");
+
+ boost::optional<const std::string&> json;
+ EXPECT_THROW(std::make_unique<Collections::Filter>(json, m),
+ std::logic_error);
+}
+
+/**
+ * Construct a valid Collections::Filter and check its public methods
+ * This creates a filter which contains a set of collections
+ */
+TEST_F(CollectionsFilterTest, filter_basic1) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "vegetable", "fruit", "meat", "dairy"]})");
+
+ std::string jsonFilter = R"({"collections":["$default", "fruit", "meat"]})";
+ boost::optional<const std::string&> json(jsonFilter);
+ Collections::Filter f(json, m);
+
+ // This is not a passthrough filter
+ EXPECT_FALSE(f.isPassthrough());
+
+ // But this filter would send the default
+ EXPECT_TRUE(f.allowDefaultCollection());
+ // and allow system events
+ EXPECT_TRUE(f.allowSystemEvents());
+
+ // The actual filter "list" only stores fruit and meat though, default is
+ // special cased via doesDefaultCollectionExist
+ EXPECT_EQ(2, f.getFilter().size());
+
+ auto list = f.getFilter();
+ EXPECT_TRUE(std::find(std::begin(list), std::end(list), "fruit") !=
+ list.end());
+ EXPECT_TRUE(std::find(std::begin(list), std::end(list), "meat") !=
+ list.end());
+}
+
+/**
+ * Construct a valid Collections::Filter and check its public methods
+ * This creates a filter which is passthrough
+ */
+TEST_F(CollectionsFilterTest, filter_basic2) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "vegetable", "fruit", "meat", "dairy"]})");
+
+ std::string jsonFilter; // empty string creates a pass through
+ boost::optional<const std::string&> json(jsonFilter);
+ Collections::Filter f(json, m);
+
+ // This is a passthrough filter
+ EXPECT_TRUE(f.isPassthrough());
+
+ // So this filter would send the default
+ EXPECT_TRUE(f.allowDefaultCollection());
+
+ // and still allow system events
+ EXPECT_TRUE(f.allowSystemEvents());
+
+ // The actual filter "list" stores nothing
+ EXPECT_EQ(0, f.getFilter().size());
+}
+
+class CollectionsVBFilterTest : public CollectionsFilterTest {};
+
+/**
+ * Create a filter with collections and check we allow what should be allowed.
+ */
+TEST_F(CollectionsVBFilterTest, basic_allow) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "vegetable", "fruit", "meat", "dairy"]})");
+
+ std::string jsonFilter = R"({"collections":["$default", "fruit", "meat"]})";
+ boost::optional<const std::string&> json(jsonFilter);
+ Collections::Filter f(json, m);
+
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ Collections::VB::Filter vbf(f, vbm);
+
+ // Yes to these guys
+ EXPECT_TRUE(vbf.allow({"anykey", DocNamespace::DefaultCollection}));
+ EXPECT_TRUE(vbf.allow({"fruit$apple", DocNamespace::Collections}));
+ EXPECT_TRUE(vbf.allow({"meat$bacon", DocNamespace::Collections}));
+
+ // No to these keys
+ EXPECT_FALSE(vbf.allow({"dairy$milk", DocNamespace::Collections}));
+ EXPECT_FALSE(vbf.allow({"vegetable$cabbage", DocNamespace::Collections}));
+
+ // There's no need yet to call the filter with DocKey's in system space, so
+ // it throws
+ EXPECT_THROW(vbf.allow({"meat$bacon", DocNamespace::System}),
+ std::invalid_argument);
+}
+
+/**
+ * Create a filter as if a legacy DCP connection would, i.e. the optional
+ * JSON filter is not initialised (because DCP open does not send a value).
+ */
+TEST_F(CollectionsVBFilterTest, legacy_filter) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$","collections":["$default", "meat"]})");
+
+ boost::optional<const std::string&> json;
+ Collections::Filter f(json, m);
+
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ Collections::VB::Filter vbf(f, vbm);
+ // Legacy would only allow default
+ EXPECT_TRUE(vbf.allow({"anykey", DocNamespace::DefaultCollection}));
+ EXPECT_FALSE(vbf.allow({"fruit$apple", DocNamespace::Collections}));
+}
+
+/**
+ * Create a passthrough filter and check it allows anything
+ */
+TEST_F(CollectionsVBFilterTest, passthrough) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$","collections":["meat"]})");
+ std::string filterJson; // empty string
+ boost::optional<const std::string&> json(filterJson);
+ Collections::Filter f(json, m);
+
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ // Everything is allowed (even junk, which isn't the filter's job to police)
+ Collections::VB::Filter vbf(f, vbm);
+ EXPECT_TRUE(vbf.allow({"anykey", DocNamespace::DefaultCollection}));
+ EXPECT_TRUE(vbf.allow({"fruit$apple", DocNamespace::Collections}));
+ EXPECT_TRUE(vbf.allow({"meat$steak", DocNamespace::Collections}));
+ EXPECT_TRUE(vbf.allow({"dairy$milk", DocNamespace::Collections}));
+ EXPECT_TRUE(vbf.allow({"JUNK!!", DocNamespace::Collections}));
+}
+
+/**
+ * Create a filter which blocks the default collection
+ */
+TEST_F(CollectionsVBFilterTest, no_default) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "vegetable", "fruit", "meat", "dairy"]})");
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ std::string jsonFilter = R"({"collections":["fruit", "meat"]})";
+ boost::optional<const std::string&> json(jsonFilter);
+ Collections::Filter f(json, m);
+
+ // Now filter!
+ Collections::VB::Filter vbf(f, vbm);
+ EXPECT_FALSE(vbf.allow({"anykey", DocNamespace::DefaultCollection}));
+ EXPECT_TRUE(vbf.allow({"fruit$apple", DocNamespace::Collections}));
+ EXPECT_TRUE(vbf.allow({"meat$steak", DocNamespace::Collections}));
+ EXPECT_FALSE(vbf.allow({"dairy$milk", DocNamespace::Collections}));
+ EXPECT_FALSE(vbf.allow({"JUNK!!", DocNamespace::Collections}));
+}
+
+/**
+ * Check we can remove collections from the filter (which live DCP may do)and
+ * check ::allow works as expected
+ */
+TEST_F(CollectionsVBFilterTest, remove1) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["vegetable", "fruit", "meat", "dairy"]})");
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ std::string jsonFilter = R"({"collections":["fruit", "meat"]})";
+ boost::optional<const std::string&> json(jsonFilter);
+
+ Collections::Filter f(json, m);
+ Collections::VB::Filter vbf(f, vbm);
+ EXPECT_TRUE(vbf.allow({"fruit$apple", DocNamespace::Collections}));
+ EXPECT_FALSE(vbf.remove("fruit"));
+ EXPECT_FALSE(vbf.allow({"fruit$apple", DocNamespace::Collections}));
+
+ EXPECT_TRUE(vbf.allow({"meat$steak", DocNamespace::Collections}));
+ EXPECT_TRUE(vbf.remove("meat"));
+ EXPECT_FALSE(vbf.allow({"meat$apple", DocNamespace::Collections}));
+}
+
+/**
+ * Check we can remove collections from the filter (which live DCP may do) and
+ * check ::allow works as expected
+ * This test includes checking we can remove $default
+ */
+TEST_F(CollectionsVBFilterTest, remove2) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "fruit", "meat", "dairy"]})");
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ std::string jsonFilter = R"({"collections":["$default", "meat"]})";
+ boost::optional<const std::string&> json(jsonFilter);
+
+ Collections::Filter f(json, m);
+ Collections::VB::Filter vbf(f, vbm);
+ EXPECT_TRUE(vbf.allow({"anykey", DocNamespace::DefaultCollection}));
+ EXPECT_FALSE(vbf.remove("$default"));
+ EXPECT_FALSE(vbf.allow({"anykey", DocNamespace::DefaultCollection}));
+
+ EXPECT_TRUE(vbf.allow({"meat$steak", DocNamespace::Collections}));
+ EXPECT_TRUE(vbf.remove("meat"));
+ EXPECT_FALSE(vbf.allow({"meat$apple", DocNamespace::Collections}));
+}
+
+std::unique_ptr<SystemEventConsumerMessage> makeTestMessage(
+ const std::string name, SystemEvent ev, const int* rev) {
+ cb::const_byte_buffer n{reinterpret_cast<const uint8_t*>(name.data()),
+ name.size()};
+ cb::const_byte_buffer r{reinterpret_cast<const uint8_t*>(rev), sizeof(int)};
+ return std::make_unique<SystemEventConsumerMessage>(
+ 0, ev, 0 /*seq*/, 0 /*vb*/, n, r);
+}
+
+/**
+ * System events are checked by a different interface (allowSystemEvent)
+ * Test that a filter allows the right events, this is a passthrough filter
+ * so everything is allowed.
+ */
+TEST_F(CollectionsVBFilterTest, system_events1) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "fruit", "meat", "dairy"]})");
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ std::string jsonFilter;
+ boost::optional<const std::string&> json(jsonFilter);
+
+ Collections::Filter f(json, m);
+ Collections::VB::Filter vbf(f, vbm);
+
+ int rev = 0;
+ // create and delete of meat is allowed by the meat filter
+ std::string name = "meat";
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::CreateCollection, &rev).get()));
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::BeginDeleteCollection, &rev)
+ .get()));
+
+ // create and delete of $default is allowed by the filter
+ name = "$default";
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::CreateCollection, &rev).get()));
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::BeginDeleteCollection, &rev)
+ .get()));
+
+ // create and delete of dairy is not allowed by the filter
+ name = "dairy";
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::CreateCollection, &rev).get()));
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::BeginDeleteCollection, &rev)
+ .get()));
+
+ // A change of separator is also allowed
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(
+ name, SystemEvent::CollectionsSeparatorChanged, &rev)
+ .get()));
+}
+
+/**
+ * System events are checked by a different interface (allowSystemEvent)
+ * Test that a filter allows the right events
+ */
+TEST_F(CollectionsVBFilterTest, system_events2) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "fruit", "meat", "dairy"]})");
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ std::string jsonFilter = R"({"collections":["$default", "meat"]})";
+ boost::optional<const std::string&> json(jsonFilter);
+
+ Collections::Filter f(json, m);
+ Collections::VB::Filter vbf(f, vbm);
+
+ int rev = 0;
+ // create and delete of meat is allowed by the meat filter
+ std::string name = "meat";
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::CreateCollection, &rev).get()));
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::BeginDeleteCollection, &rev)
+ .get()));
+
+ // create and delete of $default is allowed by the filter
+ name = "$default";
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::CreateCollection, &rev).get()));
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::BeginDeleteCollection, &rev)
+ .get()));
+
+ // create and delete of dairy is not allowed by the filter
+ name = "dairy";
+ EXPECT_FALSE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::CreateCollection, &rev).get()));
+ EXPECT_FALSE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::BeginDeleteCollection, &rev)
+ .get()));
+
+ // A change of separator is also allowed
+ EXPECT_TRUE(vbf.allowSystemEvent(
+ makeTestMessage(
+ name, SystemEvent::CollectionsSeparatorChanged, &rev)
+ .get()));
+}
+
+/**
+ * System events are checked by a different interface
+ * Test that a legacy filter denies all system events, they shouldn't be sent
+ * to legacy clients.
+ */
+TEST_F(CollectionsVBFilterTest, system_events3) {
+ Collections::Manifest m(
+ R"({"revision":0,"separator":"$",)"
+ R"("collections":["$default", "fruit", "meat", "dairy"]})");
+ Collections::VB::Manifest vbm({});
+ vbm.wlock().update(vb, m);
+
+ boost::optional<const std::string&> json;
+
+ Collections::Filter f(json, m);
+ Collections::VB::Filter vbf(f, vbm);
+
+ // All system events dropped by this empty/legacy filter
+ std::string name = "meat";
+ int rev = 0;
+ EXPECT_FALSE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::CreateCollection, &rev).get()));
+ EXPECT_FALSE(vbf.allowSystemEvent(
+ makeTestMessage(name, SystemEvent::BeginDeleteCollection, &rev)
+ .get()));
+ EXPECT_FALSE(vbf.allowSystemEvent(
+ makeTestMessage(
+ name, SystemEvent::CollectionsSeparatorChanged, &rev)
+ .get()));
+}
\ No newline at end of file
TEST(ManifestTest, defaultManifest) {
// Default construction gives the default manifest
+ // $default, rev 0 and separator of ::
Collections::Manifest manifest;
EXPECT_EQ(0, manifest.getRevision());
+ EXPECT_TRUE(manifest.doesDefaultCollectionExist());
+ EXPECT_EQ("::", manifest.getSeparator());
}
TEST(ManifestTest, getRevision) {