├── BUILD ├── CounterRecord.hh ├── CountersDecrementKafkaStoreConsumer.cpp ├── CountersDecrementKafkaStoreConsumer.h ├── CountersHandler.cpp ├── CountersHandler.h ├── CountersHandlerTest.cpp ├── CountersIncrementKafkaConsumer.cpp ├── CountersIncrementKafkaConsumer.h ├── CountersServer.cpp ├── CountersTimespans.cpp ├── CountersTimespans.h ├── IncrbyMergeOperator.h ├── LICENSE ├── README.md └── ZeroValueCompactionFilter.h /BUILD: -------------------------------------------------------------------------------- 1 | cc_binary( 2 | name = "counters", 3 | srcs = [ 4 | "CountersServer.cpp", 5 | ], 6 | deps = [ 7 | ":counters_decrement_kafka_store_consumer", 8 | ":counters_handler", 9 | ":counters_increment_kafka_consumer", 10 | "//pipeline:redis_pipeline_bootstrap", 11 | "//platform/gcloud:gcs", 12 | ], 13 | copts = [ 14 | "--std=c++14", 15 | ], 16 | ) 17 | 18 | cc_library( 19 | name = "counters_handler", 20 | srcs = [ 21 | "IncrbyMergeOperator.h", 22 | "ZeroValueCompactionFilter.h", 23 | "CountersHandler.cpp", 24 | ], 25 | hdrs = [ 26 | "CountersHandler.h", 27 | ], 28 | deps = [ 29 | "//external:boost", 30 | "//external:folly", 31 | "//external:rocksdb", 32 | "//pipeline:transactional_redis_handler", 33 | ], 34 | copts = [ 35 | "-std=c++14", 36 | ], 37 | ) 38 | 39 | cc_test( 40 | name = "counters_handler_test", 41 | srcs = [ 42 | "CountersHandlerTest.cpp" 43 | ], 44 | size = "small", 45 | deps = [ 46 | ":counters_handler", 47 | "//codec:redis_value", 48 | "//external:boost", 49 | "//external:gmock_main", 50 | "//external:gtest", 51 | "//stesting:test_helpers", 52 | ], 53 | copts = [ 54 | "-std=c++14", 55 | ], 56 | ) 57 | 58 | cc_library( 59 | name = "counters_increment_kafka_consumer", 60 | srcs = [ 61 | "CounterRecord.hh", 62 | "CountersIncrementKafkaConsumer.cpp", 63 | ], 64 | hdrs = [ 65 | "CountersIncrementKafkaConsumer.h", 66 | ], 67 | deps = [ 68 | ":counters_timespans", 69 | "//external:avro", 70 | "//external:boost", 71 | "//external:folly", 72 | "//external:glog", 73 | "//external:librdkafka", 74 | "//infra:avro_helper", 75 | "//infra/kafka:consumer", 76 | ], 77 | copts = [ 78 | "-std=c++11", 79 | ], 80 | ) 81 | 82 | cc_library( 83 | name = "counters_decrement_kafka_store_consumer", 84 | srcs = [ 85 | "CounterRecord.hh", 86 | "CountersDecrementKafkaStoreConsumer.cpp", 87 | ], 88 | hdrs = [ 89 | "CountersDecrementKafkaStoreConsumer.h", 90 | ], 91 | deps = [ 92 | ":counters_timespans", 93 | "//external:boost", 94 | "//external:folly", 95 | "//external:glog", 96 | "//infra:avro_helper", 97 | "//infra/kafka/store:consumer", 98 | "//pipeline:kafka_consumer_config", 99 | ], 100 | copts = [ 101 | "-std=c++11", 102 | ], 103 | ) 104 | 105 | cc_library( 106 | name = "counters_timespans", 107 | srcs = [ 108 | "CountersTimespans.cpp", 109 | ], 110 | hdrs = [ 111 | "CountersTimespans.h", 112 | ], 113 | copts = [ 114 | "-std=c++14", 115 | ] 116 | ) 117 | -------------------------------------------------------------------------------- /CounterRecord.hh: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | 20 | #ifndef COUNTERS_COUNTERRECORD_HH_2338794186__H_ 21 | #define COUNTERS_COUNTERRECORD_HH_2338794186__H_ 22 | 23 | 24 | #include 25 | #pragma GCC diagnostic push 26 | #pragma GCC diagnostic ignored "-Wdeprecated-declarations" 27 | #include "boost/any.hpp" 28 | #include "avro/Specific.hh" 29 | #include "avro/Encoder.hh" 30 | #include "avro/Decoder.hh" 31 | #pragma GCC diagnostic pop 32 | 33 | namespace counters { 34 | struct Counter { 35 | boost::array key; 36 | int64_t by; 37 | int64_t flags; 38 | Counter() : 39 | key(boost::array()), 40 | by(int64_t()), 41 | flags(int64_t()) 42 | { } 43 | }; 44 | 45 | } 46 | namespace avro { 47 | template<> struct codec_traits { 48 | static void encode(Encoder& e, const counters::Counter& v) { 49 | avro::encode(e, v.key); 50 | avro::encode(e, v.by); 51 | avro::encode(e, v.flags); 52 | } 53 | static void decode(Decoder& d, counters::Counter& v) { 54 | if (avro::ResolvingDecoder *rd = 55 | dynamic_cast(&d)) { 56 | const std::vector fo = rd->fieldOrder(); 57 | for (std::vector::const_iterator it = fo.begin(); 58 | it != fo.end(); ++it) { 59 | switch (*it) { 60 | case 0: 61 | avro::decode(d, v.key); 62 | break; 63 | case 1: 64 | avro::decode(d, v.by); 65 | break; 66 | case 2: 67 | avro::decode(d, v.flags); 68 | break; 69 | default: 70 | break; 71 | } 72 | } 73 | } else { 74 | avro::decode(d, v.key); 75 | avro::decode(d, v.by); 76 | // NOTE: this is hack to support newly added fields with default values. 77 | // Shouldn't need this if the C++ avro decoder is full-featured. 78 | try { 79 | avro::decode(d, v.flags); 80 | } catch (avro::Exception) {} 81 | } 82 | } 83 | }; 84 | 85 | } 86 | #endif 87 | -------------------------------------------------------------------------------- /CountersDecrementKafkaStoreConsumer.cpp: -------------------------------------------------------------------------------- 1 | #include "counters/CountersDecrementKafkaStoreConsumer.h" 2 | 3 | #include 4 | #include 5 | #include 6 | #include 7 | #include 8 | 9 | #include "boost/endian/buffers.hpp" 10 | #include "counters/CounterRecord.hh" 11 | #include "folly/Format.h" 12 | #include "glog/logging.h" 13 | #include "infra/AvroHelper.h" 14 | 15 | namespace counters { 16 | 17 | using infra::kafka::store::KafkaStoreMessage; 18 | 19 | void CountersDecrementKafkaStoreConsumer::processBatch(int timeoutMs) { 20 | ProcessingBuf buf = {}; 21 | int64_t count = consumeBatch(timeoutMs, &buf); 22 | LOG(INFO) << "Read " << count << " messages in `" << mode_ << "` mode"; 23 | commitCounts(buf); 24 | 25 | // processed delayed messages 26 | std::map delayedMsgs = std::move(buf.msgBuf); 27 | while (run() && !delayedMsgs.empty()) { 28 | // delay until the first message is due 29 | if (!delay(timeDelayMs_, delayedMsgs.begin()->second.timestamp)) { 30 | // Break early due to failed delay, e.g., the program is being terminated 31 | break; 32 | } 33 | ProcessingBuf delayedBuf = {}; 34 | for (const auto& entry : delayedMsgs) { 35 | processOne(entry.first, entry.second, &delayedBuf); 36 | } 37 | commitCounts(delayedBuf); 38 | delayedMsgs = std::move(delayedBuf.msgBuf); 39 | } 40 | } 41 | 42 | void CountersDecrementKafkaStoreConsumer::processOne(int64_t offset, const infra::kafka::store::KafkaStoreMessage& msg, 43 | void* opaque) { 44 | auto buf = static_cast(opaque); 45 | if (!buf->msgBuf.empty()) { 46 | // Assume that timestamps from kafka store messages are monotonically increasing 47 | // so once one message was buffered for delayed processing, all subsequent messages should follow 48 | buf->msgBuf.insert(std::make_pair(offset, msg)); 49 | return; 50 | } 51 | if (msg.value.is_null()) { 52 | buf->nextProcessOffset = offset + 1; 53 | LOG(ERROR) << "Message value at offset " << offset << " is null"; 54 | return; 55 | } 56 | 57 | auto valBytes = msg.value.get_bytes(); 58 | Counter record; 59 | infra::AvroHelper::decode(valBytes.data(), valBytes.size(), &record); 60 | if (nowMs() - msg.timestamp >= timeDelayMs_) { 61 | // this message is overdue, apply the count 62 | std::string key(reinterpret_cast(record.key.data()), record.key.size()); 63 | int64_t timespanFlags = record.flags ? record.flags : CountersTimespans::kDefaultTimespanFlags; 64 | if (timespanFlags & timespanMask_) { 65 | key.append(keySuffix_); 66 | buf->counts[key] -= record.by; 67 | } 68 | buf->nextProcessOffset = offset + 1; 69 | } else { 70 | // save the messaged for delayed processing 71 | buf->msgBuf.insert(std::make_pair(offset, msg)); 72 | } 73 | } 74 | 75 | void CountersDecrementKafkaStoreConsumer::commitCounts(const CountersDecrementKafkaStoreConsumer::ProcessingBuf& buf) { 76 | if (buf.counts.empty() && buf.msgBuf.empty()) { 77 | // The entire batch is empty 78 | return; 79 | } 80 | 81 | int64_t nextOffset = buf.nextProcessOffset >= 0 ? buf.nextProcessOffset : buf.msgBuf.begin()->first; 82 | rocksdb::WriteBatch writeBatch; 83 | for (const auto& entry : buf.counts) { 84 | boost::endian::big_int64_buf_t value(entry.second); 85 | writeBatch.Merge(entry.first, rocksdb::Slice(value.data(), sizeof(int64_t))); 86 | } 87 | int64_t fileOffset = buf.nextProcessOffset < nextFileOffset() ? currentFileOffset() : nextFileOffset(); 88 | CHECK(consumerHelper()->commitNextProcessKafkaAndFileOffsets(offsetKey(), nextOffset, fileOffset, &writeBatch)); 89 | // Also commit to kafka brokers only for metrics and reporting, so failure is okay 90 | if (!commitAsync()) { 91 | LOG(WARNING) << "Committing offset to kafka brokers failed"; 92 | } 93 | } 94 | 95 | bool CountersDecrementKafkaStoreConsumer::delay(int64_t delayMs, int64_t timeMs) { 96 | // Add an extra margin to delay time so that more keys are grouped for committing 97 | int64_t waitUntilMs = timeMs + delayMs + kDelayMarginMs; 98 | int64_t sleepTimeMs = waitUntilMs - nowMs(); 99 | 100 | if (sleepTimeMs <= 0) return true; 101 | DLOG(INFO) << "Sleeping for " << sleepTimeMs << "ms for delay in `" << mode_ << "` mode"; 102 | while (sleepTimeMs > 0) { 103 | if (!run()) return false; 104 | std::this_thread::sleep_for(std::min(std::chrono::milliseconds(1000), std::chrono::milliseconds(sleepTimeMs))); 105 | sleepTimeMs = waitUntilMs - nowMs(); 106 | } 107 | return true; 108 | } 109 | 110 | } // namespace counters 111 | -------------------------------------------------------------------------------- /CountersDecrementKafkaStoreConsumer.h: -------------------------------------------------------------------------------- 1 | #ifndef COUNTERS_COUNTERSDECREMENTKAFKASTORECONSUMER_H_ 2 | #define COUNTERS_COUNTERSDECREMENTKAFKASTORECONSUMER_H_ 3 | 4 | #include 5 | #include 6 | #include 7 | #include 8 | #include 9 | #include 10 | 11 | #include "boost/algorithm/string/predicate.hpp" 12 | #include "counters/CountersTimespans.h" 13 | #include "infra/kafka/store/Consumer.h" 14 | #include "infra/kafka/store/KafkaStoreMessageRecord.hh" 15 | 16 | namespace counters { 17 | 18 | class CountersDecrementKafkaStoreConsumer : public infra::kafka::store::Consumer { 19 | public: 20 | static const char* name() { 21 | return "decrement.kafka-store"; 22 | } 23 | 24 | CountersDecrementKafkaStoreConsumer(const std::string& brokerList, const std::string& objectStoreBucketName, 25 | const std::string& objectStoreObjectNamePrefix, const std::string& topic, 26 | int partition, const std::string& groupId, const std::string& offsetKey, 27 | const std::string& mode, 28 | std::shared_ptr consumerHelper, 29 | std::shared_ptr gcs) 30 | : infra::kafka::store::Consumer(brokerList, objectStoreBucketName, objectStoreObjectNamePrefix, topic, partition, 31 | groupId, offsetKey, consumerHelper, gcs), 32 | mode_(mode) { 33 | const auto it = CountersTimespans::kTimespanMap.find(mode); 34 | CHECK(it != CountersTimespans::kTimespanMap.end()) << "Unknown mode: " << mode; 35 | timeDelayMs_ = it->second.timeDelayMs; 36 | keySuffix_ = it->second.keySuffix; 37 | timespanMask_ = it->second.mask; 38 | } 39 | 40 | // Process a batch of messages and wait for the right time to decrement 41 | void processBatch(int timeoutMs) override; 42 | 43 | // Process one message from kafka store 44 | void processOne(int64_t offset, const infra::kafka::store::KafkaStoreMessage& msg, void* opaque) override; 45 | 46 | private: 47 | struct ProcessingBuf { 48 | // counts from processed messages 49 | std::unordered_map counts; 50 | // buffer for messages to be processed after a delay, keyed by kafka offset 51 | std::map msgBuf; 52 | int64_t nextProcessOffset = -1; 53 | }; 54 | 55 | // Allow a margin of error in time delay in order to group more keys in a single transaction 56 | static constexpr int64_t kDelayMarginMs = 1000; 57 | 58 | // Commit counts that are overdue 59 | void commitCounts(const ProcessingBuf& buf); 60 | 61 | // Delay timeMs for up to delayMs. Return true when delay was incurred successfully and false if interrupted. 62 | bool delay(int64_t delayMs, int64_t timeMs); 63 | 64 | const std::string mode_; 65 | int64_t timeDelayMs_; 66 | std::string keySuffix_; 67 | int64_t timespanMask_; 68 | }; 69 | 70 | } // namespace counters 71 | 72 | #endif // COUNTERS_COUNTERSDECREMENTKAFKASTORECONSUMER_H_ 73 | -------------------------------------------------------------------------------- /CountersHandler.cpp: -------------------------------------------------------------------------------- 1 | #include "counters/CountersHandler.h" 2 | 3 | #include 4 | #include 5 | #include 6 | 7 | #include "boost/endian/buffers.hpp" 8 | #include "folly/Conv.h" 9 | #include "glog/logging.h" 10 | #include "codec/RedisValue.h" 11 | #include "rocksdb/options.h" 12 | #include "rocksdb/slice.h" 13 | #include "rocksdb/status.h" 14 | 15 | namespace counters { 16 | 17 | codec::RedisValue CountersHandler::ensureCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, 18 | Context* ctx) { 19 | rocksdb::Slice key = rocksdb::Slice(cmd[1]); 20 | int64_t desiredValue = 0; 21 | try { 22 | desiredValue = folly::to(cmd[2]); 23 | } catch (std::range_error&) { 24 | return errorInvalidInteger(); 25 | } 26 | 27 | std::string value; 28 | // TODO(yunjing): support read-your-own-write by search the write batch first, when such guaranteed is needed 29 | rocksdb::Status status = db()->Get(rocksdb::ReadOptions(), key, &value); 30 | 31 | if (status.ok()) { 32 | CHECK_EQ(value.size(), sizeof(int64_t)); 33 | if (desiredValue == boost::endian::detail::load_big_endian(value.data())) { 34 | return simpleStringOk(); 35 | } 36 | return { codec::RedisValue::Type::kError, "ENSURE value different" }; 37 | } else if (status.IsNotFound()) { 38 | return { codec::RedisValue::Type::kError, "ENSURE key not found" }; 39 | } 40 | 41 | return errorResp(folly::sformat("RocksDB error: {}", status.ToString())); 42 | } 43 | 44 | codec::RedisValue CountersHandler::getCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, 45 | Context* ctx) { 46 | rocksdb::Slice key = rocksdb::Slice(cmd[1]); 47 | 48 | std::string value; 49 | // TODO(yunjing): support read-your-own-write by search the write batch first, when such guaranteed is needed 50 | rocksdb::Status status = db()->Get(rocksdb::ReadOptions(), key, &value); 51 | 52 | if (status.ok()) { 53 | CHECK_EQ(value.size(), sizeof(int64_t)); 54 | return codec::RedisValue(boost::endian::detail::load_big_endian(value.data())); 55 | } else if (status.IsNotFound()) { 56 | return codec::RedisValue::nullString(); 57 | } 58 | 59 | return errorResp(folly::sformat("RocksDB error: {}", status.ToString())); 60 | } 61 | 62 | codec::RedisValue CountersHandler::incrbyCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, 63 | Context* ctx) { 64 | rocksdb::Slice key = rocksdb::Slice(cmd[1]); 65 | int64_t delta = 0; 66 | try { 67 | delta = folly::to(cmd[2]); 68 | } catch (std::range_error&) { 69 | return errorInvalidInteger(); 70 | } 71 | 72 | boost::endian::big_int64_buf_t value(delta); 73 | // using merge to ensure atomicity with respect to multiple concurrent incrby operations 74 | writeBatch->Merge(key, rocksdb::Slice(value.data(), sizeof(int64_t))); 75 | std::string prevValue; 76 | // reading existing from database is still subject to race condition when there is a concurrent write, 77 | // but the returned value is guaranteed to be one of many legit values under certain interleaving of writes 78 | // importantly, the side-effect of the race condition is eliminated by using merge. 79 | rocksdb::Status status = db()->Get(rocksdb::ReadOptions(), key, &prevValue); 80 | 81 | if (status.ok()) { 82 | CHECK_EQ(prevValue.size(), sizeof(int64_t)); 83 | int64_t prevInt = boost::endian::detail::load_big_endian(prevValue.data()); 84 | return codec::RedisValue(prevInt + delta); 85 | } else if (status.IsNotFound()) { 86 | return codec::RedisValue(delta); 87 | } 88 | 89 | return errorResp(folly::sformat("RocksDB error: {}", status.ToString())); 90 | } 91 | 92 | codec::RedisValue CountersHandler::setCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, 93 | Context* ctx) { 94 | rocksdb::Slice key = rocksdb::Slice(cmd[1]); 95 | try { 96 | boost::endian::big_int64_buf_t value(folly::to(cmd[2])); 97 | writeBatch->Put(key, rocksdb::Slice(value.data(), sizeof(int64_t))); 98 | } catch (std::range_error&) { 99 | return errorInvalidInteger(); 100 | } 101 | 102 | return simpleStringOk(); 103 | } 104 | 105 | } // namespace counters 106 | -------------------------------------------------------------------------------- /CountersHandler.h: -------------------------------------------------------------------------------- 1 | #ifndef COUNTERS_COUNTERSHANDLER_H_ 2 | #define COUNTERS_COUNTERSHANDLER_H_ 3 | 4 | #include 5 | #include 6 | #include 7 | 8 | #include "codec/RedisValue.h" 9 | #include "counters/IncrbyMergeOperator.h" 10 | #include "counters/ZeroValueCompactionFilter.h" 11 | #include "pipeline/TransactionalRedisHandler.h" 12 | #include "rocksdb/cache.h" 13 | #include "rocksdb/filter_policy.h" 14 | #include "rocksdb/options.h" 15 | #include "rocksdb/table.h" 16 | #include "rocksdb/write_batch.h" 17 | 18 | namespace counters { 19 | 20 | class CountersHandler : public pipeline::TransactionalRedisHandler { 21 | public: 22 | CountersHandler(std::shared_ptr databaseManager, 23 | std::shared_ptr consumerHelper) 24 | : TransactionalRedisHandler(databaseManager, consumerHelper) {} 25 | 26 | static void optimizeColumnFamily(int defaultBlockCacheSizeMb, rocksdb::ColumnFamilyOptions* options) { 27 | options->compaction_filter = new ZeroValueCompactionFilter(); 28 | options->merge_operator.reset(new IncrbyMergeOperator()); 29 | // options->OptimizeForPointLookup(defaultBlockCacheSizeMb); 30 | rocksdb::BlockBasedTableOptions block_based_options; 31 | block_based_options.index_type = rocksdb::BlockBasedTableOptions::kBinarySearch; 32 | block_based_options.filter_policy.reset(rocksdb::NewBloomFilterPolicy(10)); 33 | block_based_options.block_cache = rocksdb::NewLRUCache(static_cast(defaultBlockCacheSizeMb * 1024 * 1024)); 34 | options->table_factory.reset(rocksdb::NewBlockBasedTableFactory(block_based_options)); 35 | options->memtable_prefix_bloom_size_ratio = 0.02; 36 | } 37 | 38 | const TransactionalCommandHandlerTable& getTransactionalCommandHandlerTable() const override { 39 | static const TransactionalCommandHandlerTable table(mergeWithDefaultTransactionalCommandHandlerTable({ 40 | { "ensure", { static_cast(&CountersHandler::ensureCommand), 2, 2 } }, 41 | { "get", { static_cast(&CountersHandler::getCommand), 1, 1 } }, 42 | { "incrby", { static_cast(&CountersHandler::incrbyCommand), 2, 2 } }, 43 | { "set", { static_cast(&CountersHandler::setCommand), 2, 2 } }, 44 | })); 45 | return table; 46 | } 47 | 48 | private: 49 | using TransactionalCommandHandlerFunc = pipeline::TransactionalRedisHandler::TransactionalCommandHandlerFunc; 50 | 51 | codec::RedisValue ensureCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, Context* ctx); 52 | codec::RedisValue getCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, Context* ctx); 53 | codec::RedisValue incrbyCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, Context* ctx); 54 | codec::RedisValue setCommand(const std::vector& cmd, rocksdb::WriteBatch* writeBatch, Context* ctx); 55 | }; 56 | 57 | } // namespace counters 58 | 59 | #endif // COUNTERS_COUNTERSHANDLER_H_ 60 | -------------------------------------------------------------------------------- /CountersHandlerTest.cpp: -------------------------------------------------------------------------------- 1 | #include 2 | #include 3 | #include 4 | 5 | #include "codec/RedisMessage.h" 6 | #include "counters/CountersHandler.h" 7 | #include "gmock/gmock.h" 8 | #include "gtest/gtest.h" 9 | #include "rocksdb/options.h" 10 | #include "rocksdb/slice.h" 11 | #include "rocksdb/status.h" 12 | #include "stesting/TestWithRocksDb.h" 13 | 14 | namespace counters { 15 | 16 | class CountersHandlerTest : public stesting::TestWithRocksDb { 17 | protected: 18 | CountersHandlerTest() 19 | : stesting::TestWithRocksDb({}, {{"default", CountersHandler::optimizeColumnFamily}}) {} 20 | 21 | // Counters does not support async command handling, so use default key 22 | codec::RedisMessage getRedisMessage(codec::RedisValue&& val) { 23 | return codec::RedisMessage(std::move(val)); 24 | } 25 | }; 26 | 27 | class MockCountersHandler : public CountersHandler { 28 | public: 29 | explicit MockCountersHandler(std::shared_ptr databaseManager) 30 | : CountersHandler(databaseManager, nullptr) {} 31 | 32 | MOCK_METHOD2(write, folly::Future(Context*, codec::RedisMessage)); 33 | 34 | // Counters does not support async command handling, so use default key 35 | bool handleCommand(const std::string& cmdNameLower, const std::vector& cmd, Context* ctx) { 36 | return CountersHandler::handleCommand(0L, cmdNameLower, cmd, ctx); 37 | } 38 | }; 39 | 40 | TEST_F(CountersHandlerTest, EnsureCommand) { 41 | MockCountersHandler handler(databaseManager()); 42 | 43 | // seed values 44 | boost::endian::big_int64_buf_t value1(10); 45 | db()->Put(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value1.data(), sizeof(int64_t))); 46 | 47 | // same value 48 | EXPECT_CALL(handler, 49 | write(nullptr, getRedisMessage(codec::RedisValue(codec::RedisValue::Type::kSimpleString, "OK")))) 50 | .Times(1); 51 | EXPECT_TRUE(handler.handleCommand("ensure", { "ensure", "key1", "10" }, nullptr)); 52 | 53 | // different values 54 | EXPECT_CALL( 55 | handler, 56 | write(nullptr, getRedisMessage(codec::RedisValue(codec::RedisValue::Type::kError, "ENSURE value different")))) 57 | .Times(1); 58 | EXPECT_TRUE(handler.handleCommand("ensure", { "ensure", "key1", "5" }, nullptr)); 59 | 60 | // key not found 61 | EXPECT_CALL( 62 | handler, 63 | write(nullptr, getRedisMessage(codec::RedisValue(codec::RedisValue::Type::kError, "ENSURE key not found")))) 64 | .Times(1); 65 | EXPECT_TRUE(handler.handleCommand("ensure", { "ensure", "key2", "5" }, nullptr)); 66 | 67 | // value not a valid integer 68 | EXPECT_CALL(handler, write(nullptr, getRedisMessage(codec::RedisValue(codec::RedisValue::Type::kError, 69 | "Value is not an integer or out of range")))) 70 | .Times(1); 71 | EXPECT_TRUE(handler.handleCommand("ensure", { "ensure", "key2", "a" }, nullptr)); 72 | } 73 | 74 | TEST_F(CountersHandlerTest, GetCommand) { 75 | MockCountersHandler handler(databaseManager()); 76 | 77 | // seed values 78 | boost::endian::big_int64_buf_t value1(10); 79 | db()->Put(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value1.data(), sizeof(int64_t))); 80 | 81 | // key exists 82 | EXPECT_CALL(handler, write(nullptr, getRedisMessage(codec::RedisValue(10)))).Times(1); 83 | EXPECT_TRUE(handler.handleCommand("get", { "get", "key1" }, nullptr)); 84 | 85 | // key does not exist 86 | EXPECT_CALL(handler, write(nullptr, getRedisMessage(codec::RedisValue::nullString()))).Times(1); 87 | EXPECT_TRUE(handler.handleCommand("get", { "get", "key2" }, nullptr)); 88 | } 89 | 90 | TEST_F(CountersHandlerTest, IncrbyCommand) { 91 | MockCountersHandler handler(databaseManager()); 92 | 93 | // seed values 94 | boost::endian::big_int64_buf_t value1(10); 95 | db()->Put(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value1.data(), sizeof(int64_t))); 96 | 97 | // value not a valid integer 98 | EXPECT_CALL(handler, write(nullptr, getRedisMessage(codec::RedisValue(codec::RedisValue::Type::kError, 99 | "Value is not an integer or out of range")))) 100 | .Times(1); 101 | EXPECT_TRUE(handler.handleCommand("incrby", { "incrby", "key1", "a" }, nullptr)); 102 | 103 | // key exists 104 | EXPECT_CALL(handler, write(nullptr, getRedisMessage(codec::RedisValue(15)))).Times(1); 105 | EXPECT_TRUE(handler.handleCommand("incrby", { "incrby", "key1", "5" }, nullptr)); 106 | 107 | // key does not exist 108 | EXPECT_CALL(handler, write(nullptr, getRedisMessage(codec::RedisValue(-5)))).Times(1); 109 | EXPECT_TRUE(handler.handleCommand("incrby", { "incrby", "key2", "-5" }, nullptr)); 110 | } 111 | 112 | TEST_F(CountersHandlerTest, SetCommand) { 113 | MockCountersHandler handler(databaseManager()); 114 | 115 | // value not a valid integer 116 | EXPECT_CALL(handler, write(nullptr, getRedisMessage(codec::RedisValue(codec::RedisValue::Type::kError, 117 | "Value is not an integer or out of range")))) 118 | .Times(1); 119 | EXPECT_TRUE(handler.handleCommand("set", { "set", "key1", "a" }, nullptr)); 120 | 121 | // valid integer 122 | EXPECT_CALL(handler, 123 | write(nullptr, getRedisMessage(codec::RedisValue(codec::RedisValue::Type::kSimpleString, "OK")))) 124 | .Times(1); 125 | EXPECT_TRUE(handler.handleCommand("set", { "set", "key1", "10" }, nullptr)); 126 | std::string newValue1; 127 | rocksdb::Status s1 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue1); 128 | EXPECT_TRUE(s1.ok()); 129 | int64_t intNewValue1 = boost::endian::detail::load_big_endian(newValue1.data()); 130 | EXPECT_EQ(10, intNewValue1); 131 | } 132 | 133 | TEST_F(CountersHandlerTest, ZeroValueCompactionFilter) { 134 | // no change after compaction for non-zero values 135 | boost::endian::big_int64_buf_t value1(10); 136 | db()->Put(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value1.data(), sizeof(int64_t))); 137 | db()->CompactRange(rocksdb::CompactRangeOptions(), nullptr, nullptr); 138 | std::string newValue1; 139 | rocksdb::Status s1 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue1); 140 | EXPECT_TRUE(s1.ok()); 141 | int64_t intNewValue1 = boost::endian::detail::load_big_endian(newValue1.data()); 142 | EXPECT_EQ(10, intNewValue1); 143 | 144 | // compaction deletes zero values 145 | boost::endian::big_int64_buf_t value2(0); 146 | db()->Put(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value2.data(), sizeof(int64_t))); 147 | std::string newValue2; 148 | rocksdb::Status s2 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue2); 149 | EXPECT_TRUE(s2.ok()); 150 | int64_t intNewValue2 = boost::endian::detail::load_big_endian(newValue2.data()); 151 | EXPECT_EQ(0, intNewValue2); 152 | db()->CompactRange(rocksdb::CompactRangeOptions(), nullptr, nullptr); 153 | s2 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue2); 154 | EXPECT_TRUE(s2.IsNotFound()); 155 | } 156 | 157 | TEST_F(CountersHandlerTest, IncrbyMergeOperator) { 158 | // no existing value 159 | boost::endian::big_int64_buf_t value1(10); 160 | db()->Merge(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value1.data(), sizeof(int64_t))); 161 | std::string newValue1; 162 | rocksdb::Status s1 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue1); 163 | EXPECT_TRUE(s1.ok()); 164 | int64_t intNewValue1 = boost::endian::detail::load_big_endian(newValue1.data()); 165 | EXPECT_EQ(10, intNewValue1); 166 | 167 | // with existing values 168 | boost::endian::big_int64_buf_t value2(5); 169 | db()->Merge(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value2.data(), sizeof(int64_t))); 170 | std::string newValue2; 171 | rocksdb::Status s2 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue2); 172 | EXPECT_TRUE(s2.ok()); 173 | int64_t intNewValue2 = boost::endian::detail::load_big_endian(newValue2.data()); 174 | EXPECT_EQ(15, intNewValue2); 175 | 176 | boost::endian::big_int64_buf_t value3(-16); 177 | db()->Merge(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value3.data(), sizeof(int64_t))); 178 | std::string newValue3; 179 | rocksdb::Status s3 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue3); 180 | EXPECT_TRUE(s3.ok()); 181 | int64_t intNewValue3 = boost::endian::detail::load_big_endian(newValue3.data()); 182 | EXPECT_EQ(-1, intNewValue3); 183 | 184 | boost::endian::big_int64_buf_t value4(1); 185 | db()->Merge(rocksdb::WriteOptions(), "key1", rocksdb::Slice(value4.data(), sizeof(int64_t))); 186 | std::string newValue4; 187 | rocksdb::Status s4 = db()->Get(rocksdb::ReadOptions(), "key1", &newValue4); 188 | EXPECT_TRUE(s4.ok()); 189 | int64_t intNewValue4 = boost::endian::detail::load_big_endian(newValue4.data()); 190 | EXPECT_EQ(0, intNewValue4); 191 | } 192 | 193 | } // namespace counters 194 | -------------------------------------------------------------------------------- /CountersIncrementKafkaConsumer.cpp: -------------------------------------------------------------------------------- 1 | #include "counters/CountersIncrementKafkaConsumer.h" 2 | 3 | #include 4 | #include 5 | 6 | #include "boost/endian/buffers.hpp" 7 | #include "counters/CounterRecord.hh" 8 | #include "counters/CountersTimespans.h" 9 | #include "folly/Format.h" 10 | #include "infra/AvroHelper.h" 11 | #include "rocksdb/write_batch.h" 12 | 13 | namespace counters { 14 | 15 | void CountersIncrementKafkaConsumer::processBatch(int timeoutMs) { 16 | std::unordered_map counts; 17 | int64_t prevOffset = lastProcessedOffset_; 18 | size_t count = consumeBatch(timeoutMs, &counts); 19 | if (lastProcessedOffset_ > prevOffset) { 20 | rocksdb::WriteBatch writeBatch; 21 | for (const auto& entry : counts) { 22 | boost::endian::big_int64_buf_t value(entry.second); 23 | writeBatch.Merge(entry.first, rocksdb::Slice(value.data(), sizeof(int64_t))); 24 | } 25 | CHECK(consumerHelper()->commitNextProcessOffset(offsetKey(), lastProcessedOffset_ + 1, &writeBatch)); 26 | commitAsync(); // it's okay if commit failed, since the offset in kafkadb is the source of truth 27 | DLOG(INFO) << "Batch processed " << count << " messages with " << counts.size() << " keys"; 28 | } 29 | } 30 | 31 | void CountersIncrementKafkaConsumer::processOne(const RdKafka::Message& msg, void* opaque) { 32 | auto counts = static_cast*>(opaque); 33 | Counter record; 34 | infra::AvroHelper::decode(msg.payload(), msg.len(), &record); 35 | std::string key(reinterpret_cast(record.key.data()), record.key.size()); 36 | int64_t timespanFlags = record.flags ? record.flags : CountersTimespans::kDefaultTimespanFlags; 37 | for (const auto& entry : CountersTimespans::kTimespanMap) { 38 | const auto& timespan = entry.second; 39 | if (timespanFlags & timespan.mask) { 40 | (*counts)[key + timespan.keySuffix] += record.by; 41 | } 42 | } 43 | lastProcessedOffset_ = msg.offset(); 44 | } 45 | 46 | } // namespace counters 47 | -------------------------------------------------------------------------------- /CountersIncrementKafkaConsumer.h: -------------------------------------------------------------------------------- 1 | #ifndef COUNTERS_COUNTERSINCREMENTKAFKACONSUMER_H_ 2 | #define COUNTERS_COUNTERSINCREMENTKAFKACONSUMER_H_ 3 | 4 | #include 5 | #include 6 | 7 | #include "boost/algorithm/string/predicate.hpp" 8 | #include "infra/kafka/Consumer.h" 9 | #include "librdkafka/rdkafkacpp.h" 10 | 11 | namespace counters { 12 | 13 | class CountersIncrementKafkaConsumer : public infra::kafka::Consumer { 14 | public: 15 | static const char* name() { 16 | return "increment.kafka"; 17 | } 18 | 19 | CountersIncrementKafkaConsumer(const std::string& brokerList, const std::string& topicStr, int partition, 20 | const std::string& groupId, const std::string& offsetKey, bool lowLatency, 21 | std::shared_ptr consumerHelper) 22 | : infra::kafka::Consumer(brokerList, topicStr, partition, groupId, offsetKey, lowLatency, consumerHelper), 23 | lastProcessedOffset_(RdKafka::Topic::OFFSET_INVALID) {} 24 | 25 | virtual ~CountersIncrementKafkaConsumer() {} 26 | 27 | void stop(void) override { 28 | infra::kafka::Consumer::stop(); 29 | } 30 | 31 | // Override kafka-related methods as needed 32 | // Override processBatch to allow batch-writing to rocksdb 33 | void processBatch(int timeoutMs) override; 34 | // Must override processOne to consume individual messages 35 | void processOne(const RdKafka::Message& msg, void* opaque) override; 36 | 37 | private: 38 | int64_t lastProcessedOffset_; 39 | }; 40 | 41 | } // namespace counters 42 | 43 | #endif // COUNTERS_COUNTERSINCREMENTKAFKACONSUMER_H_ 44 | -------------------------------------------------------------------------------- /CountersServer.cpp: -------------------------------------------------------------------------------- 1 | #include 2 | #include 3 | 4 | #include "counters/CountersDecrementKafkaStoreConsumer.h" 5 | #include "counters/CountersHandler.h" 6 | #include "counters/CountersIncrementKafkaConsumer.h" 7 | #include "pipeline/RedisPipelineBootstrap.h" 8 | #include "platform/gcloud/GoogleCloudStorage.h" 9 | 10 | namespace counters { 11 | 12 | static pipeline::RedisPipelineBootstrap::Config config{ 13 | redisHandlerFactory : [](pipeline::RedisPipelineBootstrap* bootstrap) -> std::shared_ptr { 14 | return std::make_shared(bootstrap->getDatabaseManager(), bootstrap->getKafkaConsumerHelper()); 15 | }, 16 | 17 | kafkaConsumerFactoryMap : 18 | {{ 19 | CountersIncrementKafkaConsumer::name(), 20 | [](const std::string& brokerList, const pipeline::KafkaConsumerConfig& consumerConfig, 21 | const std::string& offsetKey, 22 | pipeline::RedisPipelineBootstrap* bootstrap) -> std::shared_ptr { 23 | return std::make_shared( 24 | brokerList, consumerConfig.topic, consumerConfig.partition, consumerConfig.groupId, offsetKey, 25 | consumerConfig.lowLatency, bootstrap->getKafkaConsumerHelper()); 26 | }, 27 | }, 28 | { 29 | CountersDecrementKafkaStoreConsumer::name(), 30 | [](const std::string& brokerList, const pipeline::KafkaConsumerConfig& consumerConfig, 31 | const std::string& offsetKey, 32 | pipeline::RedisPipelineBootstrap* bootstrap) -> std::shared_ptr { 33 | return std::make_shared( 34 | brokerList, consumerConfig.objectStoreBucketName, consumerConfig.objectStoreObjectNamePrefix, 35 | consumerConfig.topic, consumerConfig.partition, consumerConfig.groupId, offsetKey, 36 | consumerConfig.offsetKeySuffix, bootstrap->getKafkaConsumerHelper(), 37 | std::make_shared()); 38 | }, 39 | }}, 40 | 41 | databaseManagerFactory : nullptr, 42 | 43 | scheduledTaskQueueFactory : nullptr, 44 | 45 | rocksDbCfConfiguratorMap : { 46 | { 47 | pipeline::DatabaseManager::defaultColumnFamilyName(), CountersHandler::optimizeColumnFamily, 48 | }, 49 | }, 50 | 51 | rocksDbConfigurator : nullptr, 52 | 53 | singletonRedisHandler : false, // in order to support transactions 54 | }; 55 | 56 | static auto redisPipelineBootstrap = pipeline::RedisPipelineBootstrap::create(config); 57 | 58 | } // namespace counters 59 | -------------------------------------------------------------------------------- /CountersTimespans.cpp: -------------------------------------------------------------------------------- 1 | #include "counters/CountersTimespans.h" 2 | 3 | #include 4 | 5 | namespace counters { 6 | 7 | using milliseconds = std::chrono::milliseconds; 8 | using hours = std::chrono::hours; 9 | 10 | const std::unordered_map CountersTimespans::kTimespanMap = 11 | []() -> std::unordered_map { 12 | return { 13 | {"hour", {std::chrono::duration_cast(hours(1)).count(), "H", 1L}}, 14 | {"day", {std::chrono::duration_cast(hours(24)).count(), "D", 2L}}, 15 | {"week", {std::chrono::duration_cast(hours(24 * 7)).count(), "W", 4L}}, 16 | {"month", {std::chrono::duration_cast(hours(24 * 30)).count(), "M", 8L}}, 17 | {"total", {-1L, "T", 16L}}, 18 | {"2days", {std::chrono::duration_cast(hours(24 * 2)).count(), "D2", 32L}}, 19 | {"2weeks", {std::chrono::duration_cast(hours(24 * 14)).count(), "W2", 64L}}, 20 | {"8days", {std::chrono::duration_cast(hours(24 * 8)).count(), "D8", 128L}}, 21 | {"6months", {std::chrono::duration_cast(hours(24 * 180)).count(), "M6", 256L}}, 22 | }; 23 | }(); 24 | 25 | const int64_t CountersTimespans::kDefaultTimespanFlags = 26 | CountersTimespans::kTimespanMap.at("hour").mask | CountersTimespans::kTimespanMap.at("day").mask | 27 | CountersTimespans::kTimespanMap.at("week").mask | CountersTimespans::kTimespanMap.at("month").mask; 28 | 29 | } // namespace counters 30 | -------------------------------------------------------------------------------- /CountersTimespans.h: -------------------------------------------------------------------------------- 1 | #ifndef COUNTERS_COUNTERSTIMESPANS_H_ 2 | #define COUNTERS_COUNTERSTIMESPANS_H_ 3 | 4 | #include 5 | #include 6 | #include 7 | 8 | namespace counters { 9 | 10 | class CountersTimespans { 11 | public: 12 | struct Timespan { 13 | int64_t timeDelayMs; 14 | std::string keySuffix; 15 | int64_t mask; 16 | Timespan() : timeDelayMs(-1), keySuffix(), mask(0) {} 17 | 18 | Timespan(int64_t _timeDelayMs, std::string _keySuffix, int64_t _mask) 19 | : timeDelayMs(_timeDelayMs), keySuffix(std::move(_keySuffix)), mask(_mask) {} 20 | }; 21 | 22 | // Mode -> Timespan 23 | static const std::unordered_map kTimespanMap; 24 | static const int64_t kDefaultTimespanFlags; 25 | }; 26 | 27 | } // namespace counters 28 | 29 | #endif // COUNTERS_COUNTERSTIMESPANS_H_ 30 | -------------------------------------------------------------------------------- /IncrbyMergeOperator.h: -------------------------------------------------------------------------------- 1 | #ifndef COUNTERS_INCRBYMERGEOPERATOR_H_ 2 | #define COUNTERS_INCRBYMERGEOPERATOR_H_ 3 | 4 | #include 5 | 6 | #include "boost/endian/buffers.hpp" 7 | #include "glog/logging.h" 8 | #include "rocksdb/merge_operator.h" 9 | 10 | namespace counters { 11 | 12 | using boost::endian::detail::load_big_endian; 13 | class IncrbyMergeOperator : public rocksdb::AssociativeMergeOperator { 14 | public: 15 | virtual ~IncrbyMergeOperator() {} 16 | 17 | bool Merge(const rocksdb::Slice& key, const rocksdb::Slice* existing_value, const rocksdb::Slice& value, 18 | std::string* new_value, rocksdb::Logger* logger) const override { 19 | int64_t intExistingValue = 0; 20 | if (existing_value) { 21 | CHECK_EQ(existing_value->size(), sizeof(int64_t)); 22 | intExistingValue = load_big_endian(existing_value->data()); 23 | } 24 | 25 | CHECK_EQ(value.size(), sizeof(int64_t)); 26 | int64_t intValue = load_big_endian(value.data()); 27 | 28 | boost::endian::big_int64_buf_t newValueBuf(intExistingValue + intValue); 29 | new_value->assign(newValueBuf.data(), sizeof(int64_t)); 30 | 31 | return true; 32 | } 33 | 34 | const char* Name() const override { 35 | return "CountersIncrbyMergeOperator"; 36 | } 37 | }; 38 | 39 | } // namespace counters 40 | 41 | #endif // COUNTERS_INCRBYMERGEOPERATOR_H_ 42 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright 2016 Authbox, Inc. 2 | 3 | Licensed under the Apache License, Version 2.0 (the "License"); 4 | you may not use this file except in compliance with the License. 5 | You may obtain a copy of the License at 6 | 7 | http://www.apache.org/licenses/LICENSE-2.0 8 | 9 | Unless required by applicable law or agreed to in writing, software 10 | distributed under the License is distributed on an "AS IS" BASIS, 11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | See the License for the specific language governing permissions and 13 | limitations under the License. 14 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # counters 2 | 3 | **No longer maintained:** Please note that this code is no longer being actively maintained. 4 | 5 | A counts database written in C++ that speaks Redis protocol and replicates via Kafka. 6 | 7 | ## Building from source 8 | 9 | * Ensure [Bazel](https://www.bazel.io/) is installed 10 | * Check out the [smyte-db](https://github.com/smyte/smyte-db) repo 11 | * Ensure your submodules are up-to-date: `git submodule update` 12 | * Build the project: `bazel build -c opt counters` 13 | 14 | ## Running it 15 | 16 | See `./bazel-bin/counters/counters --help` for help. More details are coming soon! 17 | -------------------------------------------------------------------------------- /ZeroValueCompactionFilter.h: -------------------------------------------------------------------------------- 1 | #ifndef COUNTERS_ZEROVALUECOMPACTIONFILTER_H_ 2 | #define COUNTERS_ZEROVALUECOMPACTIONFILTER_H_ 3 | 4 | #include 5 | 6 | #include "boost/endian/buffers.hpp" 7 | #include "rocksdb/compaction_filter.h" 8 | #include "rocksdb/slice.h" 9 | 10 | namespace counters { 11 | 12 | class ZeroValueCompactionFilter : public rocksdb::CompactionFilter { 13 | public: 14 | virtual ~ZeroValueCompactionFilter() {} 15 | 16 | bool Filter(int level, const rocksdb::Slice& key, const rocksdb::Slice& existing_value, std::string* new_value, 17 | bool* value_changed) const override { 18 | *value_changed = false; 19 | CHECK_EQ(existing_value.size(), sizeof(int64_t)); 20 | int64_t intValue = boost::endian::detail::load_big_endian(existing_value.data()); 21 | // delete the key when value is zero 22 | return intValue == 0; 23 | } 24 | 25 | const char* Name() const override { 26 | return "CountersZeroValueCompactionFilter"; 27 | } 28 | }; 29 | 30 | } // namespace counters 31 | 32 | #endif // COUNTERS_ZEROVALUECOMPACTIONFILTER_H_ 33 | --------------------------------------------------------------------------------