Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -462,7 +462,7 @@ UI_JS := $(UI_ROOT)/src/js/protos.js
UI_TS := $(UI_ROOT)/src/js/protos.d.ts
UI_PROTOS := $(UI_JS) $(UI_TS)

CPP_PROTOS := $(filter %/roachpb/metadata.proto %/roachpb/data.proto %/roachpb/internal.proto %/engine/enginepb/mvcc.proto %/engine/enginepb/mvcc3.proto %/engine/enginepb/rocksdb.proto %/hlc/legacy_timestamp.proto %/hlc/timestamp.proto %/unresolved_addr.proto,$(GO_PROTOS))
CPP_PROTOS := $(filter %/roachpb/metadata.proto %/roachpb/data.proto %/roachpb/internal.proto %/storage/preamble.proto %/engine/enginepb/mvcc.proto %/engine/enginepb/mvcc3.proto %/engine/enginepb/rocksdb.proto %/hlc/legacy_timestamp.proto %/hlc/timestamp.proto %/unresolved_addr.proto,$(GO_PROTOS))
CPP_HEADERS := $(subst $(PKG_ROOT),$(CPP_PROTO_ROOT),$(CPP_PROTOS:%.proto=%.pb.h))
CPP_SOURCES := $(subst $(PKG_ROOT),$(CPP_PROTO_ROOT),$(CPP_PROTOS:%.proto=%.pb.cc))

Expand Down
2 changes: 2 additions & 0 deletions c-deps/libroach/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,11 @@ add_library(roach
db.cc
encoding.cc
eventlistener.cc
preamble.cc
protos/roachpb/data.pb.cc
protos/roachpb/internal.pb.cc
protos/roachpb/metadata.pb.cc
protos/storage/preamble.pb.cc
protos/storage/engine/enginepb/mvcc.pb.cc
protos/storage/engine/enginepb/mvcc3.pb.cc
protos/storage/engine/enginepb/rocksdb.pb.cc
Expand Down
15 changes: 13 additions & 2 deletions c-deps/libroach/db.cc
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
#include "encoding.h"
#include "eventlistener.h"
#include "keys.h"
#include "preamble.h"

extern "C" {
static void __attribute__((noreturn)) die_missing_symbol(const char* name) {
Expand Down Expand Up @@ -89,6 +90,7 @@ struct DBEngine {
};

struct DBImpl : public DBEngine {
std::unique_ptr<PreambleHandler> preamble_handler;
std::unique_ptr<rocksdb::Env> memenv;
std::unique_ptr<rocksdb::DB> rep_deleter;
std::shared_ptr<rocksdb::Cache> block_cache;
Expand All @@ -98,8 +100,9 @@ struct DBImpl : public DBEngine {
// and Env will be deleted when the DBImpl is deleted. It is ok to
// pass NULL for the Env.
DBImpl(rocksdb::DB* r, rocksdb::Env* m, std::shared_ptr<rocksdb::Cache> bc,
std::shared_ptr<DBEventListener> event_listener)
std::shared_ptr<DBEventListener> event_listener, PreambleHandler* preamble)
: DBEngine(r),
preamble_handler(preamble),
memenv(m),
rep_deleter(r),
block_cache(bc),
Expand Down Expand Up @@ -1697,14 +1700,22 @@ DBStatus DBOpen(DBEngine **db, DBSlice dir, DBOptions db_opts) {
options.env = memenv.get();
}

PreambleHandler* preamble = nullptr;
if (db_opts.use_preamble) {
// The caller makes sure we're not an in-memory DB.
assert(dir.len != 0);
preamble = new PreambleHandler();
options.env = preamble->GetEnv(options.env);
}

rocksdb::DB *db_ptr;
rocksdb::Status status = rocksdb::DB::Open(options, ToString(dir), &db_ptr);
if (!status.ok()) {
return ToDBStatus(status);
}
*db = new DBImpl(db_ptr, memenv.release(),
db_opts.cache != nullptr ? db_opts.cache->rep : nullptr,
event_listener);
event_listener, preamble);
return kSuccess;
}

Expand Down
1 change: 1 addition & 0 deletions c-deps/libroach/include/libroach.h
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ typedef struct {
bool logging_enabled;
int num_cpu;
int max_open_files;
bool use_preamble;
} DBOptions;

// Create a new cache with the specified size.
Expand Down
120 changes: 120 additions & 0 deletions c-deps/libroach/preamble.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,120 @@
// Copyright 2017 The Cockroach Authors.
//
// 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 <arpa/inet.h>
#include "preamble.h"
#include "protos/storage/preamble.pb.h"

// Preamble length.
// WARNING: changing this will result in incompatible on-disk format.
// The preamble length must fit in a uint16_t.
const static size_t kPreambleLength = 4096;

// Blocksize for the plaintext cipher stream.
// TODO(mberhault): we need to benchmark this for a good value, but for now we use the AES::BlockSize.
const static size_t kPlaintextBlockSize = 16;

// PlaintextCipherStream implements BlockAccessCipherStream with
// no-op encrypt/decrypt operations.
class PlaintextCipherStream final : public rocksdb::BlockAccessCipherStream {
public:
PlaintextCipherStream() {}
virtual ~PlaintextCipherStream() {}

// BlockSize returns the size of each block supported by this cipher stream.
virtual size_t BlockSize() override { return kPlaintextBlockSize; }

// Encrypt blocks of data. This is a noop.
virtual rocksdb::Status Encrypt(uint64_t fileOffset, char *data, size_t dataSize) override {
return rocksdb::Status::OK();
}

// Decrypt blocks of data. This is a noop.
virtual rocksdb::Status Decrypt(uint64_t fileOffset, char *data, size_t dataSize) override {
return rocksdb::Status::OK();
}
protected:
// No-op required methods.
virtual void AllocateScratch(std::string&) override {}
virtual rocksdb::Status EncryptBlock(uint64_t blockIndex, char *data, char* scratch) override {
return rocksdb::Status::OK();
}
virtual rocksdb::Status DecryptBlock(uint64_t blockIndex, char *data, char* scratch) override {
return rocksdb::Status::OK();
}
};

size_t PreambleHandler::GetPrefixLength() {
return kPreambleLength;
}

rocksdb::Env* PreambleHandler::GetEnv(rocksdb::Env* base_env) {
return rocksdb::NewEncryptedEnv(base_env ? base_env : rocksdb::Env::Default(), this);
}

rocksdb::Status PreambleHandler::CreateNewPrefix(const std::string& fname, char *prefix, size_t prefixLength) {
// Zero-out the prefix.
memset(prefix, 0, prefixLength);

// Create a preamble proto with encryption settings.
cockroach::storage::Preamble preamble;
// Everything is plaintext for now.
preamble.set_encryption_type(cockroach::storage::Plaintext);

// Check the byte size before encoding.
int byte_size = preamble.ByteSize();

// Determine the serialized length and size of the length prefix.
assert(byte_size < UINT16_MAX);
uint16_t encoded_size = htons(byte_size);
auto num_length_bytes = sizeof(encoded_size);

if ((byte_size + num_length_bytes) > prefixLength ) {
return rocksdb::Status::Corruption("new preamble exceeds max preamble length");
}

// Write length prefix.
memcpy(prefix, &encoded_size, num_length_bytes);

// Write it to the prefix.
if (!preamble.SerializeToArray(prefix + num_length_bytes, byte_size)) {
return rocksdb::Status::Corruption("unable to write prefix");
}

return rocksdb::Status::OK();
}

rocksdb::Status PreambleHandler::CreateCipherStream(const std::string& fname, const rocksdb::EnvOptions& options, rocksdb::Slice &prefix, std::unique_ptr<rocksdb::BlockAccessCipherStream>* result) {
// Read length prefix.
uint16_t encoded_size;
auto num_length_bytes = sizeof(encoded_size);
memcpy(&encoded_size, prefix.data(), num_length_bytes);

// Convert length prefix from network byte order.
int byte_size = ntohs(encoded_size);

// Parse prefix
cockroach::storage::Preamble preamble;
if (!preamble.ParseFromArray(prefix.data() + num_length_bytes, byte_size)) {
return rocksdb::Status::Corruption("unable to parse prefix");
}

if (preamble.encryption_type() == cockroach::storage::Plaintext) {
(*result) = std::unique_ptr<rocksdb::BlockAccessCipherStream>(new PlaintextCipherStream());
} else {
return rocksdb::Status::NotSupported("unknown encryption type");
}

return rocksdb::Status::OK();
}
41 changes: 41 additions & 0 deletions c-deps/libroach/preamble.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
// Copyright 2017 The Cockroach Authors.
//
// 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.

#ifndef ROACHLIB_PREAMBLE_H
#define ROACHLIB_PREAMBLE_H

#include <rocksdb/env.h>
#include <rocksdb/env_encryption.h>

class PreambleHandler : rocksdb::EncryptionProvider {
public:

PreambleHandler() {}
virtual ~PreambleHandler() {}

// GetEnv returns an EncryptionEnv wrapped around base_env.
rocksdb::Env* GetEnv(rocksdb::Env* base_env);

// GetPrefixLength returns the preamble length.
virtual size_t GetPrefixLength() override;

// CreateNewPrefix initializes an allocated block of prefix memory for a new file.
virtual rocksdb::Status CreateNewPrefix(const std::string& fname, char *prefix, size_t prefixLength) override;

// CreateCipherStream creates a block access cipher stream for a file given name and options.
virtual rocksdb::Status CreateCipherStream(const std::string& fname, const rocksdb::EnvOptions& options,
rocksdb::Slice& prefix, std::unique_ptr<rocksdb::BlockAccessCipherStream>* result) override;
};

#endif // ROACHLIB_PREAMBLE_H
Loading