diff --git a/bindings/c/fdb_c.cpp b/bindings/c/fdb_c.cpp
index baa4caf5212..6bbc360a1fa 100644
--- a/bindings/c/fdb_c.cpp
+++ b/bindings/c/fdb_c.cpp
@@ -281,6 +281,16 @@ fdb_error_t fdb_future_get_keyvalue_array_v13(FDBFuture* f, FDBKeyValue const**
*out_count = rrr.size(););
}
+extern "C" DLLEXPORT fdb_error_t fdb_future_get_mappedkeyvalue_array(FDBFuture* f,
+ FDBMappedKeyValue const** out_kvm,
+ int* out_count,
+ fdb_bool_t* out_more) {
+ CATCH_AND_RETURN(Standalone rrr = TSAV(Standalone, f)->get();
+ *out_kvm = (FDBMappedKeyValue*)rrr.begin();
+ *out_count = rrr.size();
+ *out_more = rrr.more;);
+}
+
extern "C" DLLEXPORT fdb_error_t fdb_future_get_string_array(FDBFuture* f, const char*** out_strings, int* out_count) {
CATCH_AND_RETURN(Standalone> na = TSAV(Standalone>, f)->get();
*out_strings = (const char**)na.begin();
@@ -571,29 +581,29 @@ FDBFuture* fdb_transaction_get_range_impl(FDBTransaction* tr,
.extractPtr());
}
-FDBFuture* fdb_transaction_get_range_and_flat_map_impl(FDBTransaction* tr,
- uint8_t const* begin_key_name,
- int begin_key_name_length,
- fdb_bool_t begin_or_equal,
- int begin_offset,
- uint8_t const* end_key_name,
- int end_key_name_length,
- fdb_bool_t end_or_equal,
- int end_offset,
- uint8_t const* mapper_name,
- int mapper_name_length,
- int limit,
- int target_bytes,
- FDBStreamingMode mode,
- int iteration,
- fdb_bool_t snapshot,
- fdb_bool_t reverse) {
+FDBFuture* fdb_transaction_get_mapped_range_impl(FDBTransaction* tr,
+ uint8_t const* begin_key_name,
+ int begin_key_name_length,
+ fdb_bool_t begin_or_equal,
+ int begin_offset,
+ uint8_t const* end_key_name,
+ int end_key_name_length,
+ fdb_bool_t end_or_equal,
+ int end_offset,
+ uint8_t const* mapper_name,
+ int mapper_name_length,
+ int limit,
+ int target_bytes,
+ FDBStreamingMode mode,
+ int iteration,
+ fdb_bool_t snapshot,
+ fdb_bool_t reverse) {
FDBFuture* r = validate_and_update_parameters(limit, target_bytes, mode, iteration, reverse);
if (r != nullptr)
return r;
return (
FDBFuture*)(TXN(tr)
- ->getRangeAndFlatMap(
+ ->getMappedRange(
KeySelectorRef(KeyRef(begin_key_name, begin_key_name_length), begin_or_equal, begin_offset),
KeySelectorRef(KeyRef(end_key_name, end_key_name_length), end_or_equal, end_offset),
StringRef(mapper_name, mapper_name_length),
@@ -604,23 +614,23 @@ FDBFuture* fdb_transaction_get_range_and_flat_map_impl(FDBTransaction* tr,
}
// TODO: Support FDB_API_ADDED in generate_asm.py and then this can be replaced with fdb_api_ptr_unimpl.
-FDBFuture* fdb_transaction_get_range_and_flat_map_v699(FDBTransaction* tr,
- uint8_t const* begin_key_name,
- int begin_key_name_length,
- fdb_bool_t begin_or_equal,
- int begin_offset,
- uint8_t const* end_key_name,
- int end_key_name_length,
- fdb_bool_t end_or_equal,
- int end_offset,
- uint8_t const* mapper_name,
- int mapper_name_length,
- int limit,
- int target_bytes,
- FDBStreamingMode mode,
- int iteration,
- fdb_bool_t snapshot,
- fdb_bool_t reverse) {
+FDBFuture* fdb_transaction_get_mapped_range_v699(FDBTransaction* tr,
+ uint8_t const* begin_key_name,
+ int begin_key_name_length,
+ fdb_bool_t begin_or_equal,
+ int begin_offset,
+ uint8_t const* end_key_name,
+ int end_key_name_length,
+ fdb_bool_t end_or_equal,
+ int end_offset,
+ uint8_t const* mapper_name,
+ int mapper_name_length,
+ int limit,
+ int target_bytes,
+ FDBStreamingMode mode,
+ int iteration,
+ fdb_bool_t snapshot,
+ fdb_bool_t reverse) {
fprintf(stderr, "UNIMPLEMENTED FDB API FUNCTION\n");
abort();
}
@@ -857,7 +867,7 @@ extern "C" DLLEXPORT fdb_error_t fdb_select_api_version_impl(int runtime_version
// WARNING: use caution when implementing removed functions by calling public API functions. This can lead to
// undesired behavior when using the multi-version API. Instead, it is better to have both the removed and public
// functions call an internal implementation function. See fdb_create_database_impl for an example.
- FDB_API_CHANGED(fdb_transaction_get_range_and_flat_map, 700);
+ FDB_API_CHANGED(fdb_transaction_get_mapped_range, 700);
FDB_API_REMOVED(fdb_future_get_version, 620);
FDB_API_REMOVED(fdb_create_cluster, 610);
FDB_API_REMOVED(fdb_cluster_create_database, 610);
diff --git a/bindings/c/foundationdb/fdb_c.h b/bindings/c/foundationdb/fdb_c.h
index 9779582b751..5fc64eb7418 100644
--- a/bindings/c/foundationdb/fdb_c.h
+++ b/bindings/c/foundationdb/fdb_c.h
@@ -113,6 +113,64 @@ typedef struct keyvalue {
int value_length;
} FDBKeyValue;
#endif
+
+#pragma pack(pop)
+
+/* Memory layout of KeySelectorRef. */
+typedef struct keyselector {
+ FDBKey key;
+ /* orEqual and offset have not be tested in C binding. Just a placeholder. */
+ fdb_bool_t orEqual;
+ int offset;
+} FDBKeySelector;
+
+/* Memory layout of GetRangeReqAndResultRef. */
+typedef struct getrangereqandresult {
+ FDBKeySelector begin;
+ FDBKeySelector end;
+ FDBKeyValue* data;
+ int m_size, m_capacity;
+} FDBGetRangeReqAndResult;
+
+/* Memory layout of MappedKeyValueRef.
+
+Total 112 bytes
+- key (12 bytes)
+:74:8F:8E:5F:AE:7F:00:00
+:4A:00:00:00
+- value (12 bytes)
+:70:8F:8E:5F:AE:7F:00:00
+:00:00:00:00
+- begin selector (20 bytes)
+:30:8F:8E:5F:AE:7F:00:00
+:2D:00:00:00
+:00:7F:00:00
+:01:00:00:00
+- end selector (20 bytes)
+:EC:8E:8E:5F:AE:7F:00:00
+:2D:00:00:00
+:00:2B:3C:60
+:01:00:00:00
+- vector (16 bytes)
+:74:94:8E:5F:AE:7F:00:00
+:01:00:00:00
+:01:00:00:00
+- buffer (32 bytes)
+:00:20:D1:61:00:00:00:00
+:00:00:00:00:00:00:00:00
+:00:00:00:00:00:00:00:00
+:01:00:00:00:AE:7F:00:00
+*/
+typedef struct mappedkeyvalue {
+ FDBKey key;
+ FDBKey value;
+ /* It's complicated to map a std::variant to C. For now we assume the underlying requests are always getRange and
+ * take the shortcut. */
+ FDBGetRangeReqAndResult getRange;
+ unsigned char buffer[32];
+} FDBMappedKeyValue;
+
+#pragma pack(push, 4)
typedef struct keyrange {
const uint8_t* begin_key;
int begin_key_length;
@@ -176,6 +234,12 @@ DLLEXPORT WARN_UNUSED_RESULT fdb_error_t fdb_future_get_keyvalue_array(FDBFuture
int* out_count,
fdb_bool_t* out_more);
#endif
+
+DLLEXPORT WARN_UNUSED_RESULT fdb_error_t fdb_future_get_mappedkeyvalue_array(FDBFuture* f,
+ FDBMappedKeyValue const** out_kv,
+ int* out_count,
+ fdb_bool_t* out_more);
+
DLLEXPORT WARN_UNUSED_RESULT fdb_error_t fdb_future_get_key_array(FDBFuture* f,
FDBKey const** out_key_array,
int* out_count);
@@ -283,23 +347,23 @@ DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_range(FDBTransaction
fdb_bool_t reverse);
#endif
-DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_range_and_flat_map(FDBTransaction* tr,
- uint8_t const* begin_key_name,
- int begin_key_name_length,
- fdb_bool_t begin_or_equal,
- int begin_offset,
- uint8_t const* end_key_name,
- int end_key_name_length,
- fdb_bool_t end_or_equal,
- int end_offset,
- uint8_t const* mapper_name,
- int mapper_name_length,
- int limit,
- int target_bytes,
- FDBStreamingMode mode,
- int iteration,
- fdb_bool_t snapshot,
- fdb_bool_t reverse);
+DLLEXPORT WARN_UNUSED_RESULT FDBFuture* fdb_transaction_get_mapped_range(FDBTransaction* tr,
+ uint8_t const* begin_key_name,
+ int begin_key_name_length,
+ fdb_bool_t begin_or_equal,
+ int begin_offset,
+ uint8_t const* end_key_name,
+ int end_key_name_length,
+ fdb_bool_t end_or_equal,
+ int end_offset,
+ uint8_t const* mapper_name,
+ int mapper_name_length,
+ int limit,
+ int target_bytes,
+ FDBStreamingMode mode,
+ int iteration,
+ fdb_bool_t snapshot,
+ fdb_bool_t reverse);
DLLEXPORT void fdb_transaction_set(FDBTransaction* tr,
uint8_t const* key_name,
diff --git a/bindings/c/test/unit/fdb_api.cpp b/bindings/c/test/unit/fdb_api.cpp
index f91868e14a9..301cc5832bb 100644
--- a/bindings/c/test/unit/fdb_api.cpp
+++ b/bindings/c/test/unit/fdb_api.cpp
@@ -90,6 +90,14 @@ void Future::cancel() {
return fdb_future_get_keyvalue_array(future_, out_kv, out_count, out_more);
}
+// MappedKeyValueArrayFuture
+
+[[nodiscard]] fdb_error_t MappedKeyValueArrayFuture::get(const FDBMappedKeyValue** out_kv,
+ int* out_count,
+ fdb_bool_t* out_more) {
+ return fdb_future_get_mappedkeyvalue_array(future_, out_kv, out_count, out_more);
+}
+
// Result
Result::~Result() {
@@ -210,7 +218,7 @@ KeyValueArrayFuture Transaction::get_range(const uint8_t* begin_key_name,
reverse));
}
-KeyValueArrayFuture Transaction::get_range_and_flat_map(const uint8_t* begin_key_name,
+MappedKeyValueArrayFuture Transaction::get_mapped_range(const uint8_t* begin_key_name,
int begin_key_name_length,
fdb_bool_t begin_or_equal,
int begin_offset,
@@ -226,7 +234,7 @@ KeyValueArrayFuture Transaction::get_range_and_flat_map(const uint8_t* begin_key
int iteration,
fdb_bool_t snapshot,
fdb_bool_t reverse) {
- return KeyValueArrayFuture(fdb_transaction_get_range_and_flat_map(tr_,
+ return MappedKeyValueArrayFuture(fdb_transaction_get_mapped_range(tr_,
begin_key_name,
begin_key_name_length,
begin_or_equal,
diff --git a/bindings/c/test/unit/fdb_api.hpp b/bindings/c/test/unit/fdb_api.hpp
index a583e33fbb8..63ee9573c8d 100644
--- a/bindings/c/test/unit/fdb_api.hpp
+++ b/bindings/c/test/unit/fdb_api.hpp
@@ -135,6 +135,18 @@ class KeyValueArrayFuture : public Future {
KeyValueArrayFuture(FDBFuture* f) : Future(f) {}
};
+class MappedKeyValueArrayFuture : public Future {
+public:
+ // Call this function instead of fdb_future_get_mappedkeyvalue_array when using
+ // the MappedKeyValueArrayFuture type. Its behavior is identical to
+ // fdb_future_get_mappedkeyvalue_array.
+ fdb_error_t get(const FDBMappedKeyValue** out_kv, int* out_count, fdb_bool_t* out_more);
+
+private:
+ friend class Transaction;
+ MappedKeyValueArrayFuture(FDBFuture* f) : Future(f) {}
+};
+
class KeyRangeArrayFuture : public Future {
public:
// Call this function instead of fdb_future_get_keyrange_array when using
@@ -254,7 +266,7 @@ class Transaction final {
// WARNING: This feature is considered experimental at this time. It is only allowed when using snapshot isolation
// AND disabling read-your-writes. Returns a future which will be set to an FDBKeyValue array.
- KeyValueArrayFuture get_range_and_flat_map(const uint8_t* begin_key_name,
+ MappedKeyValueArrayFuture get_mapped_range(const uint8_t* begin_key_name,
int begin_key_name_length,
fdb_bool_t begin_or_equal,
int begin_offset,
diff --git a/bindings/c/test/unit/unit_tests.cpp b/bindings/c/test/unit/unit_tests.cpp
index 420a16dc3c9..d96f5ccdfc6 100644
--- a/bindings/c/test/unit/unit_tests.cpp
+++ b/bindings/c/test/unit/unit_tests.cpp
@@ -173,6 +173,20 @@ struct GetRangeResult {
fdb_error_t err;
};
+struct GetMappedRangeResult {
+ std::vector> // range results
+ >>
+ mkvs;
+ // True if values remain in the key range requested.
+ bool more;
+ // Set to a non-zero value if an error occurred during the transaction.
+ fdb_error_t err;
+};
+
// Helper function to get a range of kv pairs. Returns a GetRangeResult struct
// containing the results of the range read. Caller is responsible for checking
// error on failure and retrying if necessary.
@@ -225,7 +239,11 @@ GetRangeResult get_range(fdb::Transaction& tr,
return GetRangeResult{ results, out_more != 0, 0 };
}
-GetRangeResult get_range_and_flat_map(fdb::Transaction& tr,
+static inline std::string extractString(FDBKey key) {
+ return std::string((const char*)key.key, key.key_length);
+}
+
+GetMappedRangeResult get_mapped_range(fdb::Transaction& tr,
const uint8_t* begin_key_name,
int begin_key_name_length,
fdb_bool_t begin_or_equal,
@@ -242,7 +260,7 @@ GetRangeResult get_range_and_flat_map(fdb::Transaction& tr,
int iteration,
fdb_bool_t snapshot,
fdb_bool_t reverse) {
- fdb::KeyValueArrayFuture f1 = tr.get_range_and_flat_map(begin_key_name,
+ fdb::MappedKeyValueArrayFuture f1 = tr.get_mapped_range(begin_key_name,
begin_key_name_length,
begin_or_equal,
begin_offset,
@@ -261,21 +279,41 @@ GetRangeResult get_range_and_flat_map(fdb::Transaction& tr,
fdb_error_t err = wait_future(f1);
if (err) {
- return GetRangeResult{ {}, false, err };
+ return GetMappedRangeResult{ {}, false, err };
}
- const FDBKeyValue* out_kv;
+ const FDBMappedKeyValue* out_mkv;
int out_count;
fdb_bool_t out_more;
- fdb_check(f1.get(&out_kv, &out_count, &out_more));
- std::vector> results;
+ fdb_check(f1.get(&out_mkv, &out_count, &out_more));
+
+ GetMappedRangeResult result;
+ result.more = (out_more != 0);
+ result.err = 0;
+
+ // std::cout << "out_count:" << out_count << " out_more:" << out_more << " out_mkv:" << (void*)out_mkv <<
+ // std::endl;
+
for (int i = 0; i < out_count; ++i) {
- std::string key((const char*)out_kv[i].key, out_kv[i].key_length);
- std::string value((const char*)out_kv[i].value, out_kv[i].value_length);
- results.emplace_back(key, value);
+ FDBMappedKeyValue mkv = out_mkv[i];
+ auto key = extractString(mkv.key);
+ auto value = extractString(mkv.value);
+ auto begin = extractString(mkv.getRange.begin.key);
+ auto end = extractString(mkv.getRange.end.key);
+ // std::cout << "key:" << key << " value:" << value << " begin:" << begin << " end:" << end << std::endl;
+
+ std::vector> range_results;
+ for (int i = 0; i < mkv.getRange.m_size; ++i) {
+ const auto& kv = mkv.getRange.data[i];
+ std::string k((const char*)kv.key, kv.key_length);
+ std::string v((const char*)kv.value, kv.value_length);
+ range_results.emplace_back(k, v);
+ // std::cout << "[" << i << "]" << k << " -> " << v << std::endl;
+ }
+ result.mkvs.emplace_back(key, value, begin, end, range_results);
}
- return GetRangeResult{ results, out_more != 0, 0 };
+ return result;
}
// Clears all data in the database.
@@ -888,32 +926,35 @@ static Value dataOfRecord(const int i) {
static std::string indexEntryKey(const int i) {
return Tuple().append(StringRef(prefix)).append(INDEX).append(indexKey(i)).append(primaryKey(i)).pack().toString();
}
-static std::string recordKey(const int i) {
- return Tuple().append(prefix).append(RECORD).append(primaryKey(i)).pack().toString();
+static std::string recordKey(const int i, const int split) {
+ return Tuple().append(prefix).append(RECORD).append(primaryKey(i)).append(split).pack().toString();
}
-static std::string recordValue(const int i) {
- return Tuple().append(dataOfRecord(i)).pack().toString();
+static std::string recordValue(const int i, const int split) {
+ return Tuple().append(dataOfRecord(i)).append(split).pack().toString();
}
+const static int SPLIT_SIZE = 3;
std::map fillInRecords(int n) {
// Note: The user requested `prefix` should be added as the first element of the tuple that forms the key, rather
// than the prefix of the key. So we don't use key() or create_data() in this test.
std::map data;
for (int i = 0; i < n; i++) {
data[indexEntryKey(i)] = EMPTY;
- data[recordKey(i)] = recordValue(i);
+ for (int split = 0; split < SPLIT_SIZE; split++) {
+ data[recordKey(i, split)] = recordValue(i, split);
+ }
}
insert_data(db, data);
return data;
}
-GetRangeResult getIndexEntriesAndMap(int beginId, int endId, fdb::Transaction& tr) {
+GetMappedRangeResult getMappedIndexEntries(int beginId, int endId, fdb::Transaction& tr) {
std::string indexEntryKeyBegin = indexEntryKey(beginId);
std::string indexEntryKeyEnd = indexEntryKey(endId);
- std::string mapper = Tuple().append(prefix).append(RECORD).append("{K[3]}"_sr).pack().toString();
+ std::string mapper = Tuple().append(prefix).append(RECORD).append("{K[3]}"_sr).append("{...}"_sr).pack().toString();
- return get_range_and_flat_map(
+ return get_mapped_range(
tr,
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL((const uint8_t*)indexEntryKeyBegin.c_str(), indexEntryKeyBegin.size()),
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL((const uint8_t*)indexEntryKeyEnd.c_str(), indexEntryKeyEnd.size()),
@@ -923,20 +964,20 @@ GetRangeResult getIndexEntriesAndMap(int beginId, int endId, fdb::Transaction& t
/* target_bytes */ 0,
/* FDBStreamingMode */ FDB_STREAMING_MODE_WANT_ALL,
/* iteration */ 0,
- /* snapshot */ true,
+ /* snapshot */ false,
/* reverse */ 0);
}
-TEST_CASE("fdb_transaction_get_range_and_flat_map") {
- fillInRecords(20);
+TEST_CASE("fdb_transaction_get_mapped_range") {
+ const int TOTAL_RECORDS = 20;
+ fillInRecords(TOTAL_RECORDS);
fdb::Transaction tr(db);
- // get_range_and_flat_map is only support without RYW. This is a must!!!
- fdb_check(tr.set_option(FDB_TR_OPTION_READ_YOUR_WRITES_DISABLE, nullptr, 0));
+ // RYW should be enabled.
while (1) {
int beginId = 1;
int endId = 19;
- auto result = getIndexEntriesAndMap(beginId, endId, tr);
+ auto result = getMappedIndexEntries(beginId, endId, tr);
if (result.err) {
fdb::EmptyFuture f1 = tr.on_error(result.err);
@@ -945,32 +986,30 @@ TEST_CASE("fdb_transaction_get_range_and_flat_map") {
}
int expectSize = endId - beginId;
- CHECK(result.kvs.size() == expectSize);
+ CHECK(result.mkvs.size() == expectSize);
CHECK(!result.more);
int id = beginId;
- for (int i = 0; i < result.kvs.size(); i++, id++) {
- const auto& [key, value] = result.kvs[i];
- CHECK(recordKey(id).compare(key) == 0);
- CHECK(recordValue(id).compare(value) == 0);
+ for (int i = 0; i < expectSize; i++, id++) {
+ const auto& [key, value, begin, end, range_results] = result.mkvs[i];
+ CHECK(indexEntryKey(id).compare(key) == 0);
+ CHECK(EMPTY.compare(value) == 0);
+ CHECK(range_results.size() == SPLIT_SIZE);
+ for (int split = 0; split < SPLIT_SIZE; split++) {
+ auto& [k, v] = range_results[split];
+ CHECK(recordKey(id, split).compare(k) == 0);
+ CHECK(recordValue(id, split).compare(v) == 0);
+ }
}
break;
}
}
-TEST_CASE("fdb_transaction_get_range_and_flat_map get_key_values_and_map_has_more") {
- fillInRecords(2000);
- fdb::Transaction tr(db);
- fdb_check(tr.set_option(FDB_TR_OPTION_READ_YOUR_WRITES_DISABLE, nullptr, 0));
- auto result = getIndexEntriesAndMap(100, 1900, tr);
- CHECK(result.err == error_code_get_key_values_and_map_has_more);
-}
-
-TEST_CASE("fdb_transaction_get_range_and_flat_map_restricted_to_snapshot") {
+TEST_CASE("fdb_transaction_get_mapped_range_restricted_to_serializable") {
std::string mapper = Tuple().append(prefix).append(RECORD).append("{K[3]}"_sr).pack().toString();
fdb::Transaction tr(db);
fdb_check(tr.set_option(FDB_TR_OPTION_READ_YOUR_WRITES_DISABLE, nullptr, 0));
- auto result = get_range_and_flat_map(
+ auto result = get_mapped_range(
tr,
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL((const uint8_t*)indexEntryKey(0).c_str(), indexEntryKey(0).size()),
FDB_KEYSEL_FIRST_GREATER_THAN((const uint8_t*)indexEntryKey(1).c_str(), indexEntryKey(1).size()),
@@ -980,16 +1019,16 @@ TEST_CASE("fdb_transaction_get_range_and_flat_map_restricted_to_snapshot") {
/* target_bytes */ 0,
/* FDBStreamingMode */ FDB_STREAMING_MODE_WANT_ALL,
/* iteration */ 0,
- /* snapshot */ false, // Set snapshot to false
+ /* snapshot */ true, // Set snapshot to true
/* reverse */ 0);
- ASSERT(result.err == error_code_client_invalid_operation);
+ ASSERT(result.err == error_code_unsupported_operation);
}
-TEST_CASE("fdb_transaction_get_range_and_flat_map_restricted_to_ryw_disable") {
+TEST_CASE("fdb_transaction_get_mapped_range_restricted_to_ryw_enable") {
std::string mapper = Tuple().append(prefix).append(RECORD).append("{K[3]}"_sr).pack().toString();
fdb::Transaction tr(db);
- // Not set FDB_TR_OPTION_READ_YOUR_WRITES_DISABLE.
- auto result = get_range_and_flat_map(
+ fdb_check(tr.set_option(FDB_TR_OPTION_READ_YOUR_WRITES_DISABLE, nullptr, 0)); // Not disable RYW
+ auto result = get_mapped_range(
tr,
FDB_KEYSEL_FIRST_GREATER_OR_EQUAL((const uint8_t*)indexEntryKey(0).c_str(), indexEntryKey(0).size()),
FDB_KEYSEL_FIRST_GREATER_THAN((const uint8_t*)indexEntryKey(1).c_str(), indexEntryKey(1).size()),
@@ -1001,7 +1040,7 @@ TEST_CASE("fdb_transaction_get_range_and_flat_map_restricted_to_ryw_disable") {
/* iteration */ 0,
/* snapshot */ true,
/* reverse */ 0);
- ASSERT(result.err == error_code_client_invalid_operation);
+ ASSERT(result.err == error_code_unsupported_operation);
}
TEST_CASE("fdb_transaction_get_range reverse") {
diff --git a/bindings/java/CMakeLists.txt b/bindings/java/CMakeLists.txt
index 9dfb349040c..9adf24a2f75 100644
--- a/bindings/java/CMakeLists.txt
+++ b/bindings/java/CMakeLists.txt
@@ -27,6 +27,8 @@ set(JAVA_BINDING_SRCS
src/main/com/apple/foundationdb/directory/package-info.java
src/main/com/apple/foundationdb/directory/PathUtil.java
src/main/com/apple/foundationdb/DirectBufferIterator.java
+ src/main/com/apple/foundationdb/RangeResultDirectBufferIterator.java
+ src/main/com/apple/foundationdb/MappedRangeResultDirectBufferIterator.java
src/main/com/apple/foundationdb/DirectBufferPool.java
src/main/com/apple/foundationdb/FDB.java
src/main/com/apple/foundationdb/FDBDatabase.java
@@ -36,11 +38,13 @@ set(JAVA_BINDING_SRCS
src/main/com/apple/foundationdb/FutureKeyArray.java
src/main/com/apple/foundationdb/FutureResult.java
src/main/com/apple/foundationdb/FutureResults.java
+ src/main/com/apple/foundationdb/FutureMappedResults.java
src/main/com/apple/foundationdb/FutureStrings.java
src/main/com/apple/foundationdb/FutureVoid.java
src/main/com/apple/foundationdb/JNIUtil.java
src/main/com/apple/foundationdb/KeySelector.java
src/main/com/apple/foundationdb/KeyValue.java
+ src/main/com/apple/foundationdb/MappedKeyValue.java
src/main/com/apple/foundationdb/LocalityUtil.java
src/main/com/apple/foundationdb/NativeFuture.java
src/main/com/apple/foundationdb/NativeObjectWrapper.java
@@ -49,9 +53,12 @@ set(JAVA_BINDING_SRCS
src/main/com/apple/foundationdb/package-info.java
src/main/com/apple/foundationdb/Range.java
src/main/com/apple/foundationdb/RangeQuery.java
+ src/main/com/apple/foundationdb/MappedRangeQuery.java
src/main/com/apple/foundationdb/KeyArrayResult.java
src/main/com/apple/foundationdb/RangeResult.java
+ src/main/com/apple/foundationdb/MappedRangeResult.java
src/main/com/apple/foundationdb/RangeResultInfo.java
+ src/main/com/apple/foundationdb/MappedRangeResultInfo.java
src/main/com/apple/foundationdb/RangeResultSummary.java
src/main/com/apple/foundationdb/ReadTransaction.java
src/main/com/apple/foundationdb/ReadTransactionContext.java
diff --git a/bindings/java/fdbJNI.cpp b/bindings/java/fdbJNI.cpp
index b330b210b9d..1032c418e2d 100644
--- a/bindings/java/fdbJNI.cpp
+++ b/bindings/java/fdbJNI.cpp
@@ -20,6 +20,7 @@
#include
#include
+#include
#include "com_apple_foundationdb_FDB.h"
#include "com_apple_foundationdb_FDBDatabase.h"
@@ -50,10 +51,14 @@ static thread_local jmethodID g_IFutureCallback_call_methodID = JNI_NULL;
static thread_local bool is_external = false;
static jclass range_result_summary_class;
static jclass range_result_class;
+static jclass mapped_range_result_class;
+static jclass mapped_key_value_class;
static jclass string_class;
static jclass key_array_result_class;
static jmethodID key_array_result_init;
static jmethodID range_result_init;
+static jmethodID mapped_range_result_init;
+static jmethodID mapped_key_value_from_bytes;
static jmethodID range_result_summary_init;
void detachIfExternalThread(void* ignore) {
@@ -478,6 +483,127 @@ JNIEXPORT jobject JNICALL Java_com_apple_foundationdb_FutureResults_FutureResult
return result;
}
+class ExecuteOnLeave {
+ std::function func;
+
+public:
+ explicit ExecuteOnLeave(std::function func) : func(func) {}
+ ~ExecuteOnLeave() { func(); }
+};
+
+void cpBytesAndLengthInner(uint8_t*& pByte, jint*& pLength, const uint8_t* data, const int& length) {
+ *pLength = length;
+ pLength++;
+
+ memcpy(pByte, data, length);
+ pByte += length;
+}
+
+void cpBytesAndLength(uint8_t*& pByte, jint*& pLength, const FDBKey& key) {
+ cpBytesAndLengthInner(pByte, pLength, key.key, key.key_length);
+}
+
+JNIEXPORT jobject JNICALL Java_com_apple_foundationdb_FutureMappedResults_FutureMappedResults_1get(JNIEnv* jenv,
+ jobject,
+ jlong future) {
+ if (!future) {
+ throwParamNotNull(jenv);
+ return JNI_NULL;
+ }
+
+ FDBFuture* f = (FDBFuture*)future;
+
+ const FDBMappedKeyValue* kvms;
+ int count;
+ fdb_bool_t more;
+ fdb_error_t err = fdb_future_get_mappedkeyvalue_array(f, &kvms, &count, &more);
+ if (err) {
+ safeThrow(jenv, getThrowable(jenv, err));
+ return JNI_NULL;
+ }
+
+ jobjectArray mrr_values = jenv->NewObjectArray(count, mapped_key_value_class, NULL);
+ if (!mrr_values) {
+ if (!jenv->ExceptionOccurred())
+ throwOutOfMem(jenv);
+ return JNI_NULL;
+ }
+
+ for (int i = 0; i < count; i++) {
+ FDBMappedKeyValue kvm = kvms[i];
+ int kvm_count = kvm.getRange.m_size;
+
+ const int totalLengths = 4 + kvm_count * 2;
+
+ int totalBytes = kvm.key.key_length + kvm.value.key_length + kvm.getRange.begin.key.key_length +
+ kvm.getRange.end.key.key_length;
+ for (int i = 0; i < kvm_count; i++) {
+ auto kv = kvm.getRange.data[i];
+ totalBytes += kv.key_length + kv.value_length;
+ }
+
+ jbyteArray bytesArray = jenv->NewByteArray(totalBytes);
+ if (!bytesArray) {
+ if (!jenv->ExceptionOccurred())
+ throwOutOfMem(jenv);
+ return JNI_NULL;
+ }
+
+ jintArray lengthArray = jenv->NewIntArray(totalLengths);
+ if (!lengthArray) {
+ if (!jenv->ExceptionOccurred())
+ throwOutOfMem(jenv);
+ return JNI_NULL;
+ }
+
+ uint8_t* bytes_barr = (uint8_t*)jenv->GetByteArrayElements(bytesArray, JNI_NULL);
+ if (!bytes_barr) {
+ throwRuntimeEx(jenv, "Error getting handle to native resources");
+ return JNI_NULL;
+ }
+ {
+ ExecuteOnLeave e([&]() { jenv->ReleaseByteArrayElements(bytesArray, (jbyte*)bytes_barr, 0); });
+
+ jint* length_barr = jenv->GetIntArrayElements(lengthArray, JNI_NULL);
+ if (!length_barr) {
+ if (!jenv->ExceptionOccurred())
+ throwOutOfMem(jenv);
+ return JNI_NULL;
+ }
+ {
+ ExecuteOnLeave e([&]() { jenv->ReleaseIntArrayElements(lengthArray, length_barr, 0); });
+
+ uint8_t* pByte = bytes_barr;
+ jint* pLength = length_barr;
+
+ cpBytesAndLength(pByte, pLength, kvm.key);
+ cpBytesAndLength(pByte, pLength, kvm.value);
+ cpBytesAndLength(pByte, pLength, kvm.getRange.begin.key);
+ cpBytesAndLength(pByte, pLength, kvm.getRange.end.key);
+ for (int kvm_i = 0; kvm_i < kvm_count; kvm_i++) {
+ auto kv = kvm.getRange.data[kvm_i];
+ cpBytesAndLengthInner(pByte, pLength, kv.key, kv.key_length);
+ cpBytesAndLengthInner(pByte, pLength, kv.value, kv.value_length);
+ }
+ }
+ }
+ // After native arrays are released
+ jobject mkv = jenv->CallStaticObjectMethod(
+ mapped_key_value_class, mapped_key_value_from_bytes, (jbyteArray)bytesArray, (jintArray)lengthArray);
+ if (jenv->ExceptionOccurred())
+ return JNI_NULL;
+ jenv->SetObjectArrayElement(mrr_values, i, mkv);
+ if (jenv->ExceptionOccurred())
+ return JNI_NULL;
+ }
+
+ jobject mrr = jenv->NewObject(mapped_range_result_class, mapped_range_result_init, mrr_values, (jboolean)more);
+ if (jenv->ExceptionOccurred())
+ return JNI_NULL;
+
+ return mrr;
+}
+
// SOMEDAY: explore doing this more efficiently with Direct ByteBuffers
JNIEXPORT jbyteArray JNICALL Java_com_apple_foundationdb_FutureResult_FutureResult_1get(JNIEnv* jenv,
jobject,
@@ -767,23 +893,22 @@ JNIEXPORT jlong JNICALL Java_com_apple_foundationdb_FDBTransaction_Transaction_1
return (jlong)f;
}
-JNIEXPORT jlong JNICALL
-Java_com_apple_foundationdb_FDBTransaction_Transaction_1getRangeAndFlatMap(JNIEnv* jenv,
- jobject,
- jlong tPtr,
- jbyteArray keyBeginBytes,
- jboolean orEqualBegin,
- jint offsetBegin,
- jbyteArray keyEndBytes,
- jboolean orEqualEnd,
- jint offsetEnd,
- jbyteArray mapperBytes,
- jint rowLimit,
- jint targetBytes,
- jint streamingMode,
- jint iteration,
- jboolean snapshot,
- jboolean reverse) {
+JNIEXPORT jlong JNICALL Java_com_apple_foundationdb_FDBTransaction_Transaction_1getMappedRange(JNIEnv* jenv,
+ jobject,
+ jlong tPtr,
+ jbyteArray keyBeginBytes,
+ jboolean orEqualBegin,
+ jint offsetBegin,
+ jbyteArray keyEndBytes,
+ jboolean orEqualEnd,
+ jint offsetEnd,
+ jbyteArray mapperBytes,
+ jint rowLimit,
+ jint targetBytes,
+ jint streamingMode,
+ jint iteration,
+ jboolean snapshot,
+ jboolean reverse) {
if (!tPtr || !keyBeginBytes || !keyEndBytes || !mapperBytes) {
throwParamNotNull(jenv);
return 0;
@@ -814,23 +939,23 @@ Java_com_apple_foundationdb_FDBTransaction_Transaction_1getRangeAndFlatMap(JNIEn
return 0;
}
- FDBFuture* f = fdb_transaction_get_range_and_flat_map(tr,
- barrBegin,
- jenv->GetArrayLength(keyBeginBytes),
- orEqualBegin,
- offsetBegin,
- barrEnd,
- jenv->GetArrayLength(keyEndBytes),
- orEqualEnd,
- offsetEnd,
- barrMapper,
- jenv->GetArrayLength(mapperBytes),
- rowLimit,
- targetBytes,
- (FDBStreamingMode)streamingMode,
- iteration,
- snapshot,
- reverse);
+ FDBFuture* f = fdb_transaction_get_mapped_range(tr,
+ barrBegin,
+ jenv->GetArrayLength(keyBeginBytes),
+ orEqualBegin,
+ offsetBegin,
+ barrEnd,
+ jenv->GetArrayLength(keyEndBytes),
+ orEqualEnd,
+ offsetEnd,
+ barrMapper,
+ jenv->GetArrayLength(mapperBytes),
+ rowLimit,
+ targetBytes,
+ (FDBStreamingMode)streamingMode,
+ iteration,
+ snapshot,
+ reverse);
jenv->ReleaseByteArrayElements(keyBeginBytes, (jbyte*)barrBegin, JNI_ABORT);
jenv->ReleaseByteArrayElements(keyEndBytes, (jbyte*)barrEnd, JNI_ABORT);
jenv->ReleaseByteArrayElements(mapperBytes, (jbyte*)barrMapper, JNI_ABORT);
@@ -842,7 +967,6 @@ JNIEXPORT void JNICALL Java_com_apple_foundationdb_FutureResults_FutureResults_1
jlong future,
jobject jbuffer,
jint bufferCapacity) {
-
if (!future) {
throwParamNotNull(jenv);
return;
@@ -902,6 +1026,92 @@ JNIEXPORT void JNICALL Java_com_apple_foundationdb_FutureResults_FutureResults_1
}
}
+void memcpyStringInner(uint8_t* buffer, int& offset, const uint8_t* data, const int& length) {
+ memcpy(buffer + offset, &length, sizeof(jint));
+ offset += sizeof(jint);
+ memcpy(buffer + offset, data, length);
+ offset += length;
+}
+
+void memcpyString(uint8_t* buffer, int& offset, const FDBKey& key) {
+ memcpyStringInner(buffer, offset, key.key, key.key_length);
+}
+
+JNIEXPORT void JNICALL
+Java_com_apple_foundationdb_FutureMappedResults_FutureMappedResults_1getDirect(JNIEnv* jenv,
+ jobject,
+ jlong future,
+ jobject jbuffer,
+ jint bufferCapacity) {
+
+ if (!future) {
+ throwParamNotNull(jenv);
+ return;
+ }
+
+ uint8_t* buffer = (uint8_t*)jenv->GetDirectBufferAddress(jbuffer);
+ if (!buffer) {
+ if (!jenv->ExceptionOccurred())
+ throwRuntimeEx(jenv, "Error getting handle to native resources");
+ return;
+ }
+
+ FDBFuture* f = (FDBFuture*)future;
+ const FDBMappedKeyValue* kvms;
+ int count;
+ fdb_bool_t more;
+ fdb_error_t err = fdb_future_get_mappedkeyvalue_array(f, &kvms, &count, &more);
+ if (err) {
+ safeThrow(jenv, getThrowable(jenv, err));
+ return;
+ }
+
+ int totalCapacityNeeded = 2 * sizeof(jint);
+ for (int i = 0; i < count; i++) {
+ const FDBMappedKeyValue& kvm = kvms[i];
+ totalCapacityNeeded += kvm.key.key_length + kvm.value.key_length + kvm.getRange.begin.key.key_length +
+ kvm.getRange.end.key.key_length +
+ 5 * sizeof(jint); // Besides the 4 lengths above, also one for kvm_count.
+ int kvm_count = kvm.getRange.m_size;
+ for (int i = 0; i < kvm_count; i++) {
+ auto kv = kvm.getRange.data[i];
+ totalCapacityNeeded += kv.key_length + kv.value_length + 2 * sizeof(jint);
+ }
+ if (bufferCapacity < totalCapacityNeeded) {
+ count = i; /* Only fit first `i` K/V pairs */
+ more = true;
+ break;
+ }
+ }
+
+ int offset = 0;
+
+ // First copy RangeResultSummary, i.e. [keyCount, more]
+ memcpy(buffer + offset, &count, sizeof(jint));
+ offset += sizeof(jint);
+
+ memcpy(buffer + offset, &more, sizeof(jint));
+ offset += sizeof(jint);
+
+ for (int i = 0; i < count; i++) {
+ const FDBMappedKeyValue& kvm = kvms[i];
+ memcpyString(buffer, offset, kvm.key);
+ memcpyString(buffer, offset, kvm.value);
+ memcpyString(buffer, offset, kvm.getRange.begin.key);
+ memcpyString(buffer, offset, kvm.getRange.end.key);
+
+ int kvm_count = kvm.getRange.m_size;
+ memcpy(buffer + offset, &kvm_count, sizeof(jint));
+ offset += sizeof(jint);
+
+ for (int i = 0; i < kvm_count; i++) {
+ auto kv = kvm.getRange.data[i];
+ memcpyStringInner(buffer, offset, kv.key, kv.key_length);
+ memcpyStringInner(buffer, offset, kv.value, kv.value_length);
+ }
+ }
+}
+
JNIEXPORT jlong JNICALL
Java_com_apple_foundationdb_FDBTransaction_Transaction_1getEstimatedRangeSizeBytes(JNIEnv* jenv,
jobject,
@@ -1396,6 +1606,16 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) {
range_result_init = env->GetMethodID(local_range_result_class, "", "([B[IZ)V");
range_result_class = (jclass)(env)->NewGlobalRef(local_range_result_class);
+ jclass local_mapped_range_result_class = env->FindClass("com/apple/foundationdb/MappedRangeResult");
+ mapped_range_result_init =
+ env->GetMethodID(local_mapped_range_result_class, "", "([Lcom/apple/foundationdb/MappedKeyValue;Z)V");
+ mapped_range_result_class = (jclass)(env)->NewGlobalRef(local_mapped_range_result_class);
+
+ jclass local_mapped_key_value_class = env->FindClass("com/apple/foundationdb/MappedKeyValue");
+ mapped_key_value_from_bytes = env->GetStaticMethodID(
+ local_mapped_key_value_class, "fromBytes", "([B[I)Lcom/apple/foundationdb/MappedKeyValue;");
+ mapped_key_value_class = (jclass)(env)->NewGlobalRef(local_mapped_key_value_class);
+
jclass local_key_array_result_class = env->FindClass("com/apple/foundationdb/KeyArrayResult");
key_array_result_init = env->GetMethodID(local_key_array_result_class, "", "([B[I)V");
key_array_result_class = (jclass)(env)->NewGlobalRef(local_key_array_result_class);
@@ -1424,6 +1644,12 @@ void JNI_OnUnload(JavaVM* vm, void* reserved) {
if (range_result_class != JNI_NULL) {
env->DeleteGlobalRef(range_result_class);
}
+ if (mapped_range_result_class != JNI_NULL) {
+ env->DeleteGlobalRef(mapped_range_result_class);
+ }
+ if (mapped_key_value_class != JNI_NULL) {
+ env->DeleteGlobalRef(mapped_key_value_class);
+ }
if (string_class != JNI_NULL) {
env->DeleteGlobalRef(string_class);
}
diff --git a/bindings/java/src/integration/com/apple/foundationdb/RangeAndFlatMapQueryIntegrationTest.java b/bindings/java/src/integration/com/apple/foundationdb/MappedRangeQueryIntegrationTest.java
similarity index 63%
rename from bindings/java/src/integration/com/apple/foundationdb/RangeAndFlatMapQueryIntegrationTest.java
rename to bindings/java/src/integration/com/apple/foundationdb/MappedRangeQueryIntegrationTest.java
index c97ce1f7508..f8661f716ca 100644
--- a/bindings/java/src/integration/com/apple/foundationdb/RangeAndFlatMapQueryIntegrationTest.java
+++ b/bindings/java/src/integration/com/apple/foundationdb/MappedRangeQueryIntegrationTest.java
@@ -1,5 +1,5 @@
/*
- * RangeAndFlatMapQueryIntegrationTest.java
+ * MappedRangeQueryIntegrationTest.java
*
* This source file is part of the FoundationDB open source project
*
@@ -40,7 +40,7 @@
import org.junit.jupiter.api.extension.ExtendWith;
@ExtendWith(RequiresDatabase.class)
-class RangeAndFlatMapQueryIntegrationTest {
+class MappedRangeQueryIntegrationTest {
private static final FDB fdb = FDB.selectAPIVersion(710);
public String databaseArg = null;
private Database openFDB() { return fdb.open(databaseArg); }
@@ -67,16 +67,27 @@ void clearDatabase() throws Exception {
static private String indexKey(int i) { return String.format("index-key-of-record-%08d", i); }
static private String dataOfRecord(int i) { return String.format("data-of-record-%08d", i); }
- static byte[] MAPPER = Tuple.from(PREFIX, RECORD, "{K[3]}").pack();
+ static byte[] MAPPER = Tuple.from(PREFIX, RECORD, "{K[3]}", "{...}").pack();
+ static int SPLIT_SIZE = 3;
+
static private byte[] indexEntryKey(final int i) {
return Tuple.from(PREFIX, INDEX, indexKey(i), primaryKey(i)).pack();
}
- static private byte[] recordKey(final int i) { return Tuple.from(PREFIX, RECORD, primaryKey(i)).pack(); }
- static private byte[] recordValue(final int i) { return Tuple.from(dataOfRecord(i)).pack(); }
+ static private byte[] recordKeyPrefix(final int i) {
+ return Tuple.from(PREFIX, RECORD, primaryKey(i)).pack();
+ }
+ static private byte[] recordKey(final int i, final int split) {
+ return Tuple.from(PREFIX, RECORD, primaryKey(i), split).pack();
+ }
+ static private byte[] recordValue(final int i, final int split) {
+ return Tuple.from(dataOfRecord(i), split).pack();
+ }
static private void insertRecordWithIndex(final Transaction tr, final int i) {
tr.set(indexEntryKey(i), EMPTY);
- tr.set(recordKey(i), recordValue(i));
+ for (int split = 0; split < SPLIT_SIZE; split++) {
+ tr.set(recordKey(i, split), recordValue(i, split));
+ }
}
private static String getArgFromEnv() {
@@ -86,7 +97,7 @@ private static String getArgFromEnv() {
return cluster;
}
public static void main(String[] args) throws Exception {
- final RangeAndFlatMapQueryIntegrationTest test = new RangeAndFlatMapQueryIntegrationTest();
+ final MappedRangeQueryIntegrationTest test = new MappedRangeQueryIntegrationTest();
test.databaseArg = getArgFromEnv();
test.clearDatabase();
test.comparePerformance();
@@ -94,21 +105,21 @@ public static void main(String[] args) throws Exception {
}
int numRecords = 10000;
- int numQueries = 10000;
+ int numQueries = 1;
int numRecordsPerQuery = 100;
- boolean validate = false;
+ boolean validate = true;
@Test
void comparePerformance() {
FDB fdb = FDB.selectAPIVersion(710);
try (Database db = openFDB()) {
insertRecordsWithIndexes(numRecords, db);
- instrument(rangeQueryAndGet, "rangeQueryAndGet", db);
- instrument(rangeQueryAndFlatMap, "rangeQueryAndFlatMap", db);
+ instrument(rangeQueryAndThenRangeQueries, "rangeQueryAndThenRangeQueries", db);
+ instrument(mappedRangeQuery, "mappedRangeQuery", db);
}
}
private void instrument(final RangeQueryWithIndex query, final String name, final Database db) {
- System.out.printf("Starting %s (numQueries:%d, numRecordsPerQuery:%d)\n", name, numQueries, numRecordsPerQuery);
+ System.out.printf("Starting %s (numQueries:%d, numRecordsPerQuery:%d, validation:%s)\n", name, numQueries, numRecordsPerQuery, validate ? "on" : "off");
long startTime = System.currentTimeMillis();
for (int queryId = 0; queryId < numQueries; queryId++) {
int begin = ThreadLocalRandom.current().nextInt(numRecords - numRecordsPerQuery);
@@ -140,7 +151,7 @@ public interface RangeQueryWithIndex {
void run(int begin, int end, Database db);
}
- RangeQueryWithIndex rangeQueryAndGet = (int begin, int end, Database db) -> db.run(tr -> {
+ RangeQueryWithIndex rangeQueryAndThenRangeQueries = (int begin, int end, Database db) -> db.run(tr -> {
try {
List kvs = tr.getRange(KeySelector.firstGreaterOrEqual(indexEntryKey(begin)),
KeySelector.firstGreaterOrEqual(indexEntryKey(end)),
@@ -150,22 +161,25 @@ public interface RangeQueryWithIndex {
Assertions.assertEquals(end - begin, kvs.size());
// Get the records of each index entry IN PARALLEL.
- List> resultFutures = new ArrayList<>();
+ List>> resultFutures = new ArrayList<>();
// In reality, we need to get the record key by parsing the index entry key. But considering this is a
// performance test, we just ignore the returned key and simply generate it from recordKey.
for (int id = begin; id < end; id++) {
- resultFutures.add(tr.get(recordKey(id)));
+ resultFutures.add(tr.getRange(Range.startsWith(recordKeyPrefix(id)),
+ ReadTransaction.ROW_LIMIT_UNLIMITED, false, StreamingMode.WANT_ALL).asList());
}
AsyncUtil.whenAll(resultFutures).get();
if (validate) {
final Iterator indexes = kvs.iterator();
- final Iterator> records = resultFutures.iterator();
+ final Iterator>> records = resultFutures.iterator();
for (int id = begin; id < end; id++) {
Assertions.assertTrue(indexes.hasNext());
assertByteArrayEquals(indexEntryKey(id), indexes.next().getKey());
+
Assertions.assertTrue(records.hasNext());
- assertByteArrayEquals(recordValue(id), records.next().get());
+ List rangeResult = records.next().get();
+ validateRangeResult(id, rangeResult);
}
Assertions.assertFalse(indexes.hasNext());
Assertions.assertFalse(records.hasNext());
@@ -176,23 +190,32 @@ public interface RangeQueryWithIndex {
return null;
});
- RangeQueryWithIndex rangeQueryAndFlatMap = (int begin, int end, Database db) -> db.run(tr -> {
+ RangeQueryWithIndex mappedRangeQuery = (int begin, int end, Database db) -> db.run(tr -> {
try {
- tr.options().setReadYourWritesDisable();
- List kvs =
- tr.snapshot()
- .getRangeAndFlatMap(KeySelector.firstGreaterOrEqual(indexEntryKey(begin)),
- KeySelector.firstGreaterOrEqual(indexEntryKey(end)), MAPPER,
- ReadTransaction.ROW_LIMIT_UNLIMITED, false, StreamingMode.WANT_ALL)
+ List kvs =
+ tr.getMappedRange(KeySelector.firstGreaterOrEqual(indexEntryKey(begin)),
+ KeySelector.firstGreaterOrEqual(indexEntryKey(end)), MAPPER,
+ ReadTransaction.ROW_LIMIT_UNLIMITED, false, StreamingMode.WANT_ALL)
.asList()
.get();
Assertions.assertEquals(end - begin, kvs.size());
if (validate) {
- final Iterator results = kvs.iterator();
+ final Iterator results = kvs.iterator();
for (int id = begin; id < end; id++) {
Assertions.assertTrue(results.hasNext());
- assertByteArrayEquals(recordValue(id), results.next().getValue());
+ MappedKeyValue mappedKeyValue = results.next();
+ assertByteArrayEquals(indexEntryKey(id), mappedKeyValue.getKey());
+ assertByteArrayEquals(EMPTY, mappedKeyValue.getValue());
+ assertByteArrayEquals(indexEntryKey(id), mappedKeyValue.getKey());
+
+ byte[] prefix = recordKeyPrefix(id);
+ assertByteArrayEquals(prefix, mappedKeyValue.getRangeBegin());
+ prefix[prefix.length - 1] = (byte)0x01;
+ assertByteArrayEquals(prefix, mappedKeyValue.getRangeEnd());
+
+ List rangeResult = mappedKeyValue.getRangeResult();
+ validateRangeResult(id, rangeResult);
}
Assertions.assertFalse(results.hasNext());
}
@@ -202,55 +225,16 @@ public interface RangeQueryWithIndex {
return null;
});
- void assertByteArrayEquals(byte[] expected, byte[] actual) {
- Assertions.assertEquals(ByteArrayUtil.printable(expected), ByteArrayUtil.printable(actual));
+ void validateRangeResult(int id, List rangeResult) {
+ Assertions.assertEquals(rangeResult.size(), SPLIT_SIZE);
+ for (int split = 0; split < SPLIT_SIZE; split++) {
+ KeyValue keyValue = rangeResult.get(split);
+ assertByteArrayEquals(recordKey(id, split), keyValue.getKey());
+ assertByteArrayEquals(recordValue(id, split), keyValue.getValue());
+ }
}
- @Test
- void rangeAndFlatMapQueryOverMultipleRows() throws Exception {
- try (Database db = openFDB()) {
- insertRecordsWithIndexes(3, db);
-
- List expected_data_of_records = new ArrayList<>();
- for (int i = 0; i <= 1; i++) {
- expected_data_of_records.add(recordValue(i));
- }
-
- db.run(tr -> {
- // getRangeAndFlatMap is only support without RYW. This is a must!!!
- tr.options().setReadYourWritesDisable();
-
- // getRangeAndFlatMap is only supported with snapshot.
- Iterator kvs =
- tr.snapshot()
- .getRangeAndFlatMap(KeySelector.firstGreaterOrEqual(indexEntryKey(0)),
- KeySelector.firstGreaterThan(indexEntryKey(1)), MAPPER,
- ReadTransaction.ROW_LIMIT_UNLIMITED, false, StreamingMode.WANT_ALL)
- .iterator();
- Iterator expected_data_of_records_iter = expected_data_of_records.iterator();
- while (expected_data_of_records_iter.hasNext()) {
- Assertions.assertTrue(kvs.hasNext(), "iterator ended too early");
- KeyValue kv = kvs.next();
- byte[] actual_data_of_record = kv.getValue();
- byte[] expected_data_of_record = expected_data_of_records_iter.next();
-
- // System.out.println("result key:" + ByteArrayUtil.printable(kv.getKey()) + " value:" +
- // ByteArrayUtil.printable(kv.getValue())); Output:
- // result
- // key:\x02prefix\x00\x02INDEX\x00\x02index-key-of-record-0\x00\x02primary-key-of-record-0\x00
- // value:\x02data-of-record-0\x00
- // result
- // key:\x02prefix\x00\x02INDEX\x00\x02index-key-of-record-1\x00\x02primary-key-of-record-1\x00
- // value:\x02data-of-record-1\x00
-
- // For now, we don't guarantee what that the returned keys mean.
- Assertions.assertArrayEquals(expected_data_of_record, actual_data_of_record,
- "Incorrect data of record!");
- }
- Assertions.assertFalse(kvs.hasNext(), "Iterator returned too much data");
-
- return null;
- });
- }
+ void assertByteArrayEquals(byte[] expected, byte[] actual) {
+ Assertions.assertEquals(ByteArrayUtil.printable(expected), ByteArrayUtil.printable(actual));
}
}
diff --git a/bindings/java/src/junit/com/apple/foundationdb/FakeFDBTransaction.java b/bindings/java/src/junit/com/apple/foundationdb/FakeFDBTransaction.java
index 0c5a121c64a..f557b6f7a16 100644
--- a/bindings/java/src/junit/com/apple/foundationdb/FakeFDBTransaction.java
+++ b/bindings/java/src/junit/com/apple/foundationdb/FakeFDBTransaction.java
@@ -89,8 +89,6 @@ public CompletableFuture get(byte[] key) {
@Override
protected FutureResults getRange_internal(KeySelector begin, KeySelector end,
- // TODO: map is not supported in FakeFDBTransaction yet.
- byte[] mapper, // Nullable
int rowLimit, int targetBytes, int streamingMode, int iteration,
boolean isSnapshot, boolean reverse) {
numRangeCalls++;
diff --git a/bindings/java/src/main/com/apple/foundationdb/DirectBufferIterator.java b/bindings/java/src/main/com/apple/foundationdb/DirectBufferIterator.java
index 289f0f0432e..ac2eb2b2dc0 100644
--- a/bindings/java/src/main/com/apple/foundationdb/DirectBufferIterator.java
+++ b/bindings/java/src/main/com/apple/foundationdb/DirectBufferIterator.java
@@ -32,11 +32,11 @@
* The serialization format of result is =>
* [int keyCount, boolean more, ListOf<(int keyLen, int valueLen, byte[] key, byte[] value)>]
*/
-class DirectBufferIterator implements Iterator, AutoCloseable {
- private ByteBuffer byteBuffer;
- private int current = 0;
- private int keyCount = -1;
- private boolean more = false;
+abstract class DirectBufferIterator implements AutoCloseable {
+ protected ByteBuffer byteBuffer;
+ protected int current = 0;
+ protected int keyCount = -1;
+ protected boolean more = false;
public DirectBufferIterator(ByteBuffer buffer) {
byteBuffer = buffer;
@@ -55,31 +55,11 @@ public boolean hasResultReady() {
return keyCount > -1;
}
- @Override
public boolean hasNext() {
assert (hasResultReady());
return current < keyCount;
}
- @Override
- public KeyValue next() {
- assert (hasResultReady()); // Must be called once its ready.
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
-
- final int keyLen = byteBuffer.getInt();
- final int valueLen = byteBuffer.getInt();
- byte[] key = new byte[keyLen];
- byteBuffer.get(key);
-
- byte[] value = new byte[valueLen];
- byteBuffer.get(value);
-
- current += 1;
- return new KeyValue(key, value);
- }
-
public ByteBuffer getBuffer() {
return byteBuffer;
}
diff --git a/bindings/java/src/main/com/apple/foundationdb/FDBTransaction.java b/bindings/java/src/main/com/apple/foundationdb/FDBTransaction.java
index 8a30280a4d7..65a1e9f2548 100644
--- a/bindings/java/src/main/com/apple/foundationdb/FDBTransaction.java
+++ b/bindings/java/src/main/com/apple/foundationdb/FDBTransaction.java
@@ -92,12 +92,10 @@ public CompletableFuture getRangeSplitPoints(Range range, long c
}
@Override
- public AsyncIterable getRangeAndFlatMap(KeySelector begin, KeySelector end, byte[] mapper, int limit,
- boolean reverse, StreamingMode mode) {
- if (mapper == null) {
- throw new IllegalArgumentException("Mapper must be non-null");
- }
- return new RangeQuery(FDBTransaction.this, true, begin, end, mapper, limit, reverse, mode, eventKeeper);
+ public AsyncIterable getMappedRange(KeySelector begin, KeySelector end, byte[] mapper,
+ int limit, boolean reverse, StreamingMode mode) {
+
+ throw new UnsupportedOperationException("getMappedRange is only supported in serializable");
}
///////////////////
@@ -348,9 +346,12 @@ public CompletableFuture getRangeSplitPoints(Range range, long c
}
@Override
- public AsyncIterable getRangeAndFlatMap(KeySelector begin, KeySelector end, byte[] mapper, int limit,
- boolean reverse, StreamingMode mode) {
- throw new UnsupportedOperationException("getRangeAndFlatMap is only supported in snapshot");
+ public AsyncIterable getMappedRange(KeySelector begin, KeySelector end, byte[] mapper,
+ int limit, boolean reverse, StreamingMode mode) {
+ if (mapper == null) {
+ throw new IllegalArgumentException("Mapper must be non-null");
+ }
+ return new MappedRangeQuery(FDBTransaction.this, false, begin, end, mapper, limit, reverse, mode, eventKeeper);
}
///////////////////
@@ -431,7 +432,6 @@ public Database getDatabase() {
// Users of this function must close the returned FutureResults when finished
protected FutureResults getRange_internal(KeySelector begin, KeySelector end,
- byte[] mapper, // Nullable
int rowLimit, int targetBytes, int streamingMode, int iteration,
boolean isSnapshot, boolean reverse) {
if (eventKeeper != null) {
@@ -443,14 +443,33 @@ protected FutureResults getRange_internal(KeySelector begin, KeySelector end,
" -- range get: (%s, %s) limit: %d, bytes: %d, mode: %d, iteration: %d, snap: %s, reverse %s",
begin.toString(), end.toString(), rowLimit, targetBytes, streamingMode,
iteration, Boolean.toString(isSnapshot), Boolean.toString(reverse)));*/
- return new FutureResults(
- mapper == null
- ? Transaction_getRange(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(), end.getKey(),
- end.orEqual(), end.getOffset(), rowLimit, targetBytes, streamingMode,
- iteration, isSnapshot, reverse)
- : Transaction_getRangeAndFlatMap(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(),
- end.getKey(), end.orEqual(), end.getOffset(), mapper, rowLimit,
- targetBytes, streamingMode, iteration, isSnapshot, reverse),
+ return new FutureResults(Transaction_getRange(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(),
+ end.getKey(), end.orEqual(), end.getOffset(), rowLimit,
+ targetBytes, streamingMode, iteration, isSnapshot, reverse),
+ FDB.instance().isDirectBufferQueriesEnabled(), executor, eventKeeper);
+ } finally {
+ pointerReadLock.unlock();
+ }
+ }
+
+ // Users of this function must close the returned FutureResults when finished
+ protected FutureMappedResults getMappedRange_internal(KeySelector begin, KeySelector end,
+ byte[] mapper, // Nullable
+ int rowLimit, int targetBytes, int streamingMode,
+ int iteration, boolean isSnapshot, boolean reverse) {
+ if (eventKeeper != null) {
+ eventKeeper.increment(Events.JNI_CALL);
+ }
+ pointerReadLock.lock();
+ try {
+ /*System.out.println(String.format(
+ " -- range get: (%s, %s) limit: %d, bytes: %d, mode: %d, iteration: %d, snap: %s, reverse %s",
+ begin.toString(), end.toString(), rowLimit, targetBytes, streamingMode,
+ iteration, Boolean.toString(isSnapshot), Boolean.toString(reverse)));*/
+ return new FutureMappedResults(
+ Transaction_getMappedRange(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(),
+ end.getKey(), end.orEqual(), end.getOffset(), mapper, rowLimit,
+ targetBytes, streamingMode, iteration, isSnapshot, reverse),
FDB.instance().isDirectBufferQueriesEnabled(), executor, eventKeeper);
} finally {
pointerReadLock.unlock();
@@ -790,7 +809,7 @@ private native long Transaction_getRange(long cPtr,
byte[] keyEnd, boolean orEqualEnd, int offsetEnd,
int rowLimit, int targetBytes, int streamingMode, int iteration,
boolean isSnapshot, boolean reverse);
- private native long Transaction_getRangeAndFlatMap(long cPtr, byte[] keyBegin, boolean orEqualBegin,
+ private native long Transaction_getMappedRange(long cPtr, byte[] keyBegin, boolean orEqualBegin,
int offsetBegin, byte[] keyEnd, boolean orEqualEnd,
int offsetEnd,
byte[] mapper, // Nonnull
diff --git a/bindings/java/src/main/com/apple/foundationdb/FutureMappedResults.java b/bindings/java/src/main/com/apple/foundationdb/FutureMappedResults.java
new file mode 100644
index 00000000000..8d4436d6f7f
--- /dev/null
+++ b/bindings/java/src/main/com/apple/foundationdb/FutureMappedResults.java
@@ -0,0 +1,87 @@
+/*
+ * FutureMappedResults.java
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2022 Apple Inc. and the FoundationDB project 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.
+ */
+
+package com.apple.foundationdb;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+
+import com.apple.foundationdb.EventKeeper.Events;
+
+class FutureMappedResults extends NativeFuture {
+ private final EventKeeper eventKeeper;
+ FutureMappedResults(long cPtr, boolean enableDirectBufferQueries, Executor executor, EventKeeper eventKeeper) {
+ super(cPtr);
+ registerMarshalCallback(executor);
+ this.enableDirectBufferQueries = enableDirectBufferQueries;
+ this.eventKeeper = eventKeeper;
+ }
+
+ @Override
+ protected void postMarshal(MappedRangeResultInfo rri) {
+ // We can't close because this class actually marshals on-demand
+ }
+
+ @Override
+ protected MappedRangeResultInfo getIfDone_internal(long cPtr) throws FDBException {
+ if (eventKeeper != null) {
+ eventKeeper.increment(Events.JNI_CALL);
+ }
+ FDBException err = Future_getError(cPtr);
+
+ if (err != null && !err.isSuccess()) {
+ throw err;
+ }
+
+ return new MappedRangeResultInfo(this);
+ }
+
+ public MappedRangeResult getResults() {
+ ByteBuffer buffer = enableDirectBufferQueries ? DirectBufferPool.getInstance().poll() : null;
+ if (buffer != null && eventKeeper != null) {
+ eventKeeper.increment(Events.RANGE_QUERY_DIRECT_BUFFER_HIT);
+ eventKeeper.increment(Events.JNI_CALL);
+ } else if (eventKeeper != null) {
+ eventKeeper.increment(Events.RANGE_QUERY_DIRECT_BUFFER_MISS);
+ eventKeeper.increment(Events.JNI_CALL);
+ }
+
+ try {
+ pointerReadLock.lock();
+ if (buffer != null) {
+ try (MappedRangeResultDirectBufferIterator directIterator =
+ new MappedRangeResultDirectBufferIterator(buffer)) {
+ FutureMappedResults_getDirect(getPtr(), directIterator.getBuffer(),
+ directIterator.getBuffer().capacity());
+ return new MappedRangeResult(directIterator);
+ }
+ } else {
+ return FutureMappedResults_get(getPtr());
+ }
+ } finally {
+ pointerReadLock.unlock();
+ }
+ }
+
+ private boolean enableDirectBufferQueries = false;
+
+ private native MappedRangeResult FutureMappedResults_get(long cPtr) throws FDBException;
+ private native void FutureMappedResults_getDirect(long cPtr, ByteBuffer buffer, int capacity) throws FDBException;
+}
diff --git a/bindings/java/src/main/com/apple/foundationdb/FutureResults.java b/bindings/java/src/main/com/apple/foundationdb/FutureResults.java
index d941a065820..989c3766200 100644
--- a/bindings/java/src/main/com/apple/foundationdb/FutureResults.java
+++ b/bindings/java/src/main/com/apple/foundationdb/FutureResults.java
@@ -66,7 +66,7 @@ public RangeResult getResults() {
try {
pointerReadLock.lock();
if (buffer != null) {
- try (DirectBufferIterator directIterator = new DirectBufferIterator(buffer)) {
+ try (RangeResultDirectBufferIterator directIterator = new RangeResultDirectBufferIterator(buffer)) {
FutureResults_getDirect(getPtr(), directIterator.getBuffer(), directIterator.getBuffer().capacity());
return new RangeResult(directIterator);
}
diff --git a/bindings/java/src/main/com/apple/foundationdb/KeyValue.java b/bindings/java/src/main/com/apple/foundationdb/KeyValue.java
index b3594c59030..e1e22b76a36 100644
--- a/bindings/java/src/main/com/apple/foundationdb/KeyValue.java
+++ b/bindings/java/src/main/com/apple/foundationdb/KeyValue.java
@@ -20,6 +20,8 @@
package com.apple.foundationdb;
+import com.apple.foundationdb.tuple.ByteArrayUtil;
+
import java.util.Arrays;
/**
@@ -77,4 +79,13 @@ public boolean equals(Object obj) {
public int hashCode() {
return 17 + (37 * Arrays.hashCode(key) + Arrays.hashCode(value));
}
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder("KeyValue{");
+ sb.append("key=").append(ByteArrayUtil.printable(key));
+ sb.append(", value=").append(ByteArrayUtil.printable(value));
+ sb.append('}');
+ return sb.toString();
+ }
}
\ No newline at end of file
diff --git a/bindings/java/src/main/com/apple/foundationdb/MappedKeyValue.java b/bindings/java/src/main/com/apple/foundationdb/MappedKeyValue.java
new file mode 100644
index 00000000000..71bad2caa98
--- /dev/null
+++ b/bindings/java/src/main/com/apple/foundationdb/MappedKeyValue.java
@@ -0,0 +1,96 @@
+/*
+ * MappedKeyValue.java
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project 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.
+ */
+
+package com.apple.foundationdb;
+
+import com.apple.foundationdb.tuple.ByteArrayUtil;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+public class MappedKeyValue extends KeyValue {
+ private final byte[] rangeBegin;
+ private final byte[] rangeEnd;
+ private final List rangeResult;
+
+ MappedKeyValue(byte[] key, byte[] value, byte[] rangeBegin, byte[] rangeEnd, List rangeResult) {
+ super(key, value);
+ this.rangeBegin = rangeBegin;
+ this.rangeEnd = rangeEnd;
+ this.rangeResult = rangeResult;
+ }
+
+ public byte[] getRangeBegin() { return rangeBegin; }
+
+ public byte[] getRangeEnd() { return rangeEnd; }
+
+ public List getRangeResult() { return rangeResult; }
+
+ public static MappedKeyValue fromBytes(byte[] bytes, int[] lengths) {
+ // Lengths include: key, value, rangeBegin, rangeEnd, count * (underlying key, underlying value)
+ if (lengths.length < 4) {
+ throw new IllegalArgumentException("There needs to be at least 4 lengths to cover the metadata");
+ }
+
+ Offset offset = new Offset();
+ byte[] key = takeBytes(offset, bytes, lengths);
+ byte[] value = takeBytes(offset, bytes, lengths);
+ byte[] rangeBegin = takeBytes(offset, bytes, lengths);
+ byte[] rangeEnd = takeBytes(offset, bytes, lengths);
+
+ if ((lengths.length - 4) % 2 != 0) {
+ throw new IllegalArgumentException("There needs to be an even number of lengths!");
+ }
+ int count = (lengths.length - 4) / 2;
+ List rangeResult = new ArrayList<>(count);
+ for (int i = 0; i < count; i++) {
+ byte[] k = takeBytes(offset, bytes, lengths);
+ byte[] v = takeBytes(offset, bytes, lengths);
+ rangeResult.add(new KeyValue(k, v));
+ }
+ return new MappedKeyValue(key, value, rangeBegin, rangeEnd, rangeResult);
+ }
+
+ static class Offset {
+ int bytes = 0;
+ int lengths = 0;
+ }
+
+ static byte[] takeBytes(Offset offset, byte[] bytes, int[] lengths) {
+ int len = lengths[offset.lengths];
+ byte[] b = new byte[len];
+ System.arraycopy(bytes, offset.bytes, b, 0, len);
+ offset.lengths++;
+ offset.bytes += len;
+ return b;
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder("MappedKeyValue{");
+ sb.append("rangeBegin=").append(ByteArrayUtil.printable(rangeBegin));
+ sb.append(", rangeEnd=").append(ByteArrayUtil.printable(rangeEnd));
+ sb.append(", rangeResult=").append(rangeResult);
+ sb.append('}');
+ return super.toString() + "->" + sb.toString();
+ }
+}
\ No newline at end of file
diff --git a/bindings/java/src/main/com/apple/foundationdb/MappedRangeQuery.java b/bindings/java/src/main/com/apple/foundationdb/MappedRangeQuery.java
new file mode 100644
index 00000000000..9de3753ec32
--- /dev/null
+++ b/bindings/java/src/main/com/apple/foundationdb/MappedRangeQuery.java
@@ -0,0 +1,333 @@
+/*
+ * RangeQuery.java
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2018 Apple Inc. and the FoundationDB project 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.
+ */
+
+package com.apple.foundationdb;
+
+import com.apple.foundationdb.EventKeeper.Events;
+import com.apple.foundationdb.async.AsyncIterable;
+import com.apple.foundationdb.async.AsyncIterator;
+import com.apple.foundationdb.async.AsyncUtil;
+
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiConsumer;
+
+// TODO: Share code with RangeQuery?
+/**
+ * Represents a query against FoundationDB for a range of keys. The
+ * result of this query can be iterated over in a blocking fashion with a call to
+ * {@link #iterator()} (as specified by {@link Iterable}).
+ * If the calling program uses an asynchronous paradigm, a non-blocking
+ * {@link AsyncIterator} is returned from {@link #iterator()}. Both of these
+ * constructions will not begin to query the database until the first call to
+ * {@code hasNext()}. As the query uses its {@link Transaction} of origin to fetch
+ * all the data, the use of this query object must not span more than a few seconds.
+ *
+ *
NOTE: although resulting {@code Iterator}s do support the {@code remove()}
+ * operation, the remove is not durable until {@code commit()} on the {@code Transaction}
+ * that yielded this query returns true
.
+ */
+class MappedRangeQuery implements AsyncIterable {
+ private final FDBTransaction tr;
+ private final KeySelector begin;
+ private final KeySelector end;
+ private final byte[] mapper; // Nonnull
+ private final boolean snapshot;
+ private final int rowLimit;
+ private final boolean reverse;
+ private final StreamingMode streamingMode;
+ private final EventKeeper eventKeeper;
+
+ MappedRangeQuery(FDBTransaction transaction, boolean isSnapshot, KeySelector begin, KeySelector end, byte[] mapper,
+ int rowLimit, boolean reverse, StreamingMode streamingMode, EventKeeper eventKeeper) {
+ this.tr = transaction;
+ this.begin = begin;
+ this.end = end;
+ this.mapper = mapper;
+ this.snapshot = isSnapshot;
+ this.rowLimit = rowLimit;
+ this.reverse = reverse;
+ this.streamingMode = streamingMode;
+ this.eventKeeper = eventKeeper;
+ }
+
+ /**
+ * Returns all the results from the range requested as a {@code List}. If there were no
+ * limits on the original query and there is a large amount of data in the database
+ * this call could use a very large amount of memory.
+ *
+ * @return a {@code CompletableFuture} that will be set to the contents of the database
+ * constrained by the query parameters.
+ */
+ @Override
+ public CompletableFuture> asList() {
+ StreamingMode mode = this.streamingMode;
+ if (mode == StreamingMode.ITERATOR) mode = (this.rowLimit == 0) ? StreamingMode.WANT_ALL : StreamingMode.EXACT;
+
+ // if the streaming mode is EXACT, try and grab things as one chunk
+ if (mode == StreamingMode.EXACT) {
+
+ FutureMappedResults range =
+ tr.getMappedRange_internal(this.begin, this.end, this.mapper, this.rowLimit, 0,
+ StreamingMode.EXACT.code(), 1, this.snapshot, this.reverse);
+ return range.thenApply(result -> result.get().values).whenComplete((result, e) -> range.close());
+ }
+
+ // If the streaming mode is not EXACT, simply collect the results of an
+ // iteration into a list
+ return AsyncUtil.collect(
+ new MappedRangeQuery(tr, snapshot, begin, end, mapper, rowLimit, reverse, mode, eventKeeper),
+ tr.getExecutor());
+ }
+
+ /**
+ * Returns an {@code Iterator} over the results of this query against FoundationDB.
+ *
+ * @return an {@code Iterator} over type {@code MappedKeyValue}.
+ */
+ @Override
+ public AsyncRangeIterator iterator() {
+ return new AsyncRangeIterator(this.rowLimit, this.reverse, this.streamingMode);
+ }
+
+ private class AsyncRangeIterator implements AsyncIterator {
+ // immutable aspects of this iterator
+ private final boolean rowsLimited;
+ private final boolean reverse;
+ private final StreamingMode streamingMode;
+
+ // There is the chance for parallelism in the two "chunks" for fetched data
+ private MappedRangeResult chunk = null;
+ private MappedRangeResult nextChunk = null;
+ private boolean fetchOutstanding = false;
+ private byte[] prevKey = null;
+ private int index = 0;
+ private int iteration = 0;
+ private KeySelector begin;
+ private KeySelector end;
+
+ private int rowsRemaining;
+
+ private FutureMappedResults fetchingChunk;
+ private CompletableFuture nextFuture;
+ private boolean isCancelled = false;
+
+ private AsyncRangeIterator(int rowLimit, boolean reverse, StreamingMode streamingMode) {
+ this.begin = MappedRangeQuery.this.begin;
+ this.end = MappedRangeQuery.this.end;
+ this.rowsLimited = rowLimit != 0;
+ this.rowsRemaining = rowLimit;
+ this.reverse = reverse;
+ this.streamingMode = streamingMode;
+
+ startNextFetch();
+ }
+
+ private synchronized boolean mainChunkIsTheLast() { return !chunk.more || (rowsLimited && rowsRemaining < 1); }
+
+ class FetchComplete implements BiConsumer {
+ final FutureMappedResults fetchingChunk;
+ final CompletableFuture promise;
+
+ FetchComplete(FutureMappedResults fetch, CompletableFuture promise) {
+ this.fetchingChunk = fetch;
+ this.promise = promise;
+ }
+
+ @Override
+ public void accept(MappedRangeResultInfo data, Throwable error) {
+ try {
+ if (error != null) {
+ if (eventKeeper != null) {
+ eventKeeper.increment(Events.RANGE_QUERY_CHUNK_FAILED);
+ }
+ promise.completeExceptionally(error);
+ if (error instanceof Error) {
+ throw(Error) error;
+ }
+
+ return;
+ }
+
+ final MappedRangeResult rangeResult = data.get();
+ final RangeResultSummary summary = rangeResult.getSummary();
+ if (summary.lastKey == null) {
+ promise.complete(Boolean.FALSE);
+ return;
+ }
+
+ synchronized (MappedRangeQuery.AsyncRangeIterator.this) {
+ fetchOutstanding = false;
+
+ // adjust the total number of rows we should ever fetch
+ rowsRemaining -= summary.keyCount;
+
+ // set up the next fetch
+ if (reverse) {
+ end = KeySelector.firstGreaterOrEqual(summary.lastKey);
+ } else {
+ begin = KeySelector.firstGreaterThan(summary.lastKey);
+ }
+
+ // If this is the first fetch or the main chunk is exhausted
+ if (chunk == null || index == chunk.values.size()) {
+ nextChunk = null;
+ chunk = rangeResult;
+ index = 0;
+ } else {
+ nextChunk = rangeResult;
+ }
+ }
+
+ promise.complete(Boolean.TRUE);
+ } finally {
+ fetchingChunk.close();
+ }
+ }
+ }
+
+ private synchronized void startNextFetch() {
+ if (fetchOutstanding)
+ throw new IllegalStateException("Reentrant call not allowed"); // This can not be called reentrantly
+ if (isCancelled) return;
+
+ if (chunk != null && mainChunkIsTheLast()) return;
+
+ fetchOutstanding = true;
+ nextChunk = null;
+
+ nextFuture = new CompletableFuture<>();
+ final long sTime = System.nanoTime();
+ fetchingChunk = tr.getMappedRange_internal(begin, end, mapper, rowsLimited ? rowsRemaining : 0, 0,
+ streamingMode.code(), ++iteration, snapshot, reverse);
+
+ BiConsumer cons = new FetchComplete(fetchingChunk, nextFuture);
+ if (eventKeeper != null) {
+ eventKeeper.increment(Events.RANGE_QUERY_FETCHES);
+ cons = cons.andThen((r, t) -> {
+ eventKeeper.timeNanos(Events.RANGE_QUERY_FETCH_TIME_NANOS, System.nanoTime() - sTime);
+ });
+ }
+
+ fetchingChunk.whenComplete(cons);
+ }
+
+ @Override
+ public synchronized CompletableFuture onHasNext() {
+ if (isCancelled) throw new CancellationException();
+
+ // This will only happen before the first fetch has completed
+ if (chunk == null) {
+ return nextFuture;
+ }
+
+ // We have a chunk and are still working though it
+ if (index < chunk.values.size()) {
+ return AsyncUtil.READY_TRUE;
+ }
+
+ // If we are at the end of the current chunk there is either:
+ // - no more data -or-
+ // - we are already fetching the next block
+ return mainChunkIsTheLast() ? AsyncUtil.READY_FALSE : nextFuture;
+ }
+
+ @Override
+ public boolean hasNext() {
+ return onHasNext().join();
+ }
+
+ @Override
+ public MappedKeyValue next() {
+ CompletableFuture nextFuture;
+ synchronized (this) {
+ if (isCancelled) throw new CancellationException();
+
+ // at least the first chunk has been fetched and there is at least one
+ // available result
+ if (chunk != null && index < chunk.values.size()) {
+ // If this is the first call to next() on a chunk, then we will want to
+ // start fetching the data for the next block
+ boolean initialNext = index == 0;
+
+ MappedKeyValue result = chunk.values.get(index);
+ prevKey = result.getKey();
+ index++;
+
+ if (eventKeeper != null) {
+ // We record the BYTES_FETCHED here, rather than at a lower level,
+ // because some parts of the construction of a MappedRangeResult occur underneath
+ // the JNI boundary, and we don't want to pass the eventKeeper down there
+ // (note: account for the length fields as well when recording the bytes
+ // fetched)
+ eventKeeper.count(Events.BYTES_FETCHED, result.getKey().length + result.getValue().length + 8);
+ eventKeeper.increment(Events.RANGE_QUERY_RECORDS_FETCHED);
+ }
+
+ // If this is the first call to next() on a chunk there cannot
+ // be another waiting, since we could not have issued a request
+ assert (!(initialNext && nextChunk != null));
+
+ // we are at the end of the current chunk and there is more to be had already
+ if (index == chunk.values.size() && nextChunk != null) {
+ index = 0;
+ chunk = nextChunk;
+ nextChunk = null;
+ }
+
+ if (initialNext) {
+ startNextFetch();
+ }
+
+ return result;
+ }
+
+ nextFuture = onHasNext();
+ }
+
+ // If there was no result ready then we need to wait on the future
+ // and return the proper result, throwing if there are no more elements
+ return nextFuture
+ .thenApply(hasNext -> {
+ if (hasNext) {
+ return next();
+ }
+ throw new NoSuchElementException();
+ })
+ .join();
+ }
+
+ @Override
+ public synchronized void remove() {
+ if (prevKey == null) throw new IllegalStateException("No value has been fetched from database");
+
+ tr.clear(prevKey);
+ }
+
+ @Override
+ public synchronized void cancel() {
+ isCancelled = true;
+ nextFuture.cancel(true);
+ fetchingChunk.cancel(true);
+ }
+ }
+}
diff --git a/bindings/java/src/main/com/apple/foundationdb/MappedRangeResult.java b/bindings/java/src/main/com/apple/foundationdb/MappedRangeResult.java
new file mode 100644
index 00000000000..e629984d7a6
--- /dev/null
+++ b/bindings/java/src/main/com/apple/foundationdb/MappedRangeResult.java
@@ -0,0 +1,64 @@
+/*
+ * MappedRangeResult.java
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2022 Apple Inc. and the FoundationDB project 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.
+ */
+
+package com.apple.foundationdb;
+
+import com.apple.foundationdb.tuple.ByteArrayUtil;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+class MappedRangeResult {
+ final List values;
+ final boolean more;
+
+ public MappedRangeResult(MappedKeyValue[] values, boolean more) {
+ this.values = Arrays.asList(values);
+ this.more = more;
+ }
+
+ MappedRangeResult(MappedRangeResultDirectBufferIterator iterator) {
+ iterator.readResultsSummary();
+ more = iterator.hasMore();
+
+ int count = iterator.count();
+ values = new ArrayList<>(count);
+
+ for (int i = 0; i < count; ++i) {
+ values.add(iterator.next());
+ }
+ }
+
+ public RangeResultSummary getSummary() {
+ final int keyCount = values.size();
+ final byte[] lastKey = keyCount > 0 ? values.get(keyCount - 1).getKey() : null;
+ return new RangeResultSummary(lastKey, keyCount, more);
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder("MappedRangeResult{");
+ sb.append("values=").append(values);
+ sb.append(", more=").append(more);
+ sb.append('}');
+ return sb.toString();
+ }
+}
diff --git a/bindings/java/src/main/com/apple/foundationdb/MappedRangeResultDirectBufferIterator.java b/bindings/java/src/main/com/apple/foundationdb/MappedRangeResultDirectBufferIterator.java
new file mode 100644
index 00000000000..169cef42e0f
--- /dev/null
+++ b/bindings/java/src/main/com/apple/foundationdb/MappedRangeResultDirectBufferIterator.java
@@ -0,0 +1,71 @@
+/*
+ * MappedRangeResultDirectBufferIterator.java
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2015-2022 Apple Inc. and the FoundationDB project 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.
+ */
+
+package com.apple.foundationdb;
+
+import java.io.Closeable;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+/**
+ * Holds the direct buffer that is shared with JNI wrapper.
+ */
+class MappedRangeResultDirectBufferIterator extends DirectBufferIterator implements Iterator {
+
+ MappedRangeResultDirectBufferIterator(ByteBuffer buffer) { super(buffer); }
+
+ @Override
+ public boolean hasNext() {
+ return super.hasNext();
+ }
+
+ @Override
+ public MappedKeyValue next() {
+ assert (hasResultReady()); // Must be called once its ready.
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+
+ final byte[] key = getString();
+ final byte[] value = getString();
+ final byte[] rangeBegin = getString();
+ final byte[] rangeEnd = getString();
+ final int rangeResultSize = byteBuffer.getInt();
+ List rangeResult = new ArrayList();
+ for (int i = 0; i < rangeResultSize; i++) {
+ final byte[] k = getString();
+ final byte[] v = getString();
+ rangeResult.add(new KeyValue(k, v));
+ }
+ current += 1;
+ return new MappedKeyValue(key, value, rangeBegin, rangeEnd, rangeResult);
+ }
+
+ private byte[] getString() {
+ final int len = byteBuffer.getInt();
+ byte[] s = new byte[len];
+ byteBuffer.get(s);
+ return s;
+ }
+}
\ No newline at end of file
diff --git a/bindings/java/src/main/com/apple/foundationdb/MappedRangeResultInfo.java b/bindings/java/src/main/com/apple/foundationdb/MappedRangeResultInfo.java
new file mode 100644
index 00000000000..6f5ce16d2b1
--- /dev/null
+++ b/bindings/java/src/main/com/apple/foundationdb/MappedRangeResultInfo.java
@@ -0,0 +1,29 @@
+/*
+ * MappedRangeResultInfo.java
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2013-2022 Apple Inc. and the FoundationDB project 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.
+ */
+
+package com.apple.foundationdb;
+
+class MappedRangeResultInfo {
+ MappedRangeResult get() { return f.getResults(); }
+
+ MappedRangeResultInfo(FutureMappedResults f) { this.f = f; }
+
+ private FutureMappedResults f;
+}
diff --git a/bindings/java/src/main/com/apple/foundationdb/RangeQuery.java b/bindings/java/src/main/com/apple/foundationdb/RangeQuery.java
index f91b00471ad..77e62427357 100644
--- a/bindings/java/src/main/com/apple/foundationdb/RangeQuery.java
+++ b/bindings/java/src/main/com/apple/foundationdb/RangeQuery.java
@@ -49,19 +49,17 @@ class RangeQuery implements AsyncIterable {
private final FDBTransaction tr;
private final KeySelector begin;
private final KeySelector end;
- private final byte[] mapper; // Nullable
private final boolean snapshot;
private final int rowLimit;
private final boolean reverse;
private final StreamingMode streamingMode;
private final EventKeeper eventKeeper;
- RangeQuery(FDBTransaction transaction, boolean isSnapshot, KeySelector begin, KeySelector end, byte[] mapper,
- int rowLimit, boolean reverse, StreamingMode streamingMode, EventKeeper eventKeeper) {
+ RangeQuery(FDBTransaction transaction, boolean isSnapshot, KeySelector begin, KeySelector end, int rowLimit,
+ boolean reverse, StreamingMode streamingMode, EventKeeper eventKeeper) {
this.tr = transaction;
this.begin = begin;
this.end = end;
- this.mapper = mapper;
this.snapshot = isSnapshot;
this.rowLimit = rowLimit;
this.reverse = reverse;
@@ -69,12 +67,6 @@ class RangeQuery implements AsyncIterable {
this.eventKeeper = eventKeeper;
}
- // RangeQueryAndFlatMap
- RangeQuery(FDBTransaction transaction, boolean isSnapshot, KeySelector begin, KeySelector end, int rowLimit,
- boolean reverse, StreamingMode streamingMode, EventKeeper eventKeeper) {
- this(transaction, isSnapshot, begin, end, null, rowLimit, reverse, streamingMode, eventKeeper);
- }
-
/**
* Returns all the results from the range requested as a {@code List}. If there were no
* limits on the original query and there is a large amount of data in the database
@@ -92,7 +84,7 @@ public CompletableFuture> asList() {
// if the streaming mode is EXACT, try and grab things as one chunk
if(mode == StreamingMode.EXACT) {
- FutureResults range = tr.getRange_internal(this.begin, this.end, this.mapper, this.rowLimit, 0,
+ FutureResults range = tr.getRange_internal(this.begin, this.end, this.rowLimit, 0,
StreamingMode.EXACT.code(), 1, this.snapshot, this.reverse);
return range.thenApply(result -> result.get().values)
.whenComplete((result, e) -> range.close());
@@ -100,7 +92,7 @@ public CompletableFuture> asList() {
// If the streaming mode is not EXACT, simply collect the results of an
// iteration into a list
- return AsyncUtil.collect(new RangeQuery(tr, snapshot, begin, end, mapper, rowLimit, reverse, mode, eventKeeper),
+ return AsyncUtil.collect(new RangeQuery(tr, snapshot, begin, end, rowLimit, reverse, mode, eventKeeper),
tr.getExecutor());
}
@@ -229,8 +221,8 @@ private synchronized void startNextFetch() {
nextFuture = new CompletableFuture<>();
final long sTime = System.nanoTime();
- fetchingChunk = tr.getRange_internal(begin, end, mapper, rowsLimited ? rowsRemaining : 0, 0,
- streamingMode.code(), ++iteration, snapshot, reverse);
+ fetchingChunk = tr.getRange_internal(begin, end, rowsLimited ? rowsRemaining : 0, 0, streamingMode.code(),
+ ++iteration, snapshot, reverse);
BiConsumer cons = new FetchComplete(fetchingChunk,nextFuture);
if(eventKeeper!=null){
diff --git a/bindings/java/src/main/com/apple/foundationdb/RangeResult.java b/bindings/java/src/main/com/apple/foundationdb/RangeResult.java
index c20c1556e0c..7c9ffaf25e3 100644
--- a/bindings/java/src/main/com/apple/foundationdb/RangeResult.java
+++ b/bindings/java/src/main/com/apple/foundationdb/RangeResult.java
@@ -58,7 +58,7 @@ class RangeResult {
this.more = more;
}
- RangeResult(DirectBufferIterator iterator) {
+ RangeResult(RangeResultDirectBufferIterator iterator) {
iterator.readResultsSummary();
more = iterator.hasMore();
diff --git a/bindings/java/src/main/com/apple/foundationdb/RangeResultDirectBufferIterator.java b/bindings/java/src/main/com/apple/foundationdb/RangeResultDirectBufferIterator.java
new file mode 100644
index 00000000000..c2c66e12276
--- /dev/null
+++ b/bindings/java/src/main/com/apple/foundationdb/RangeResultDirectBufferIterator.java
@@ -0,0 +1,62 @@
+/*
+ * RangeResultDirectBufferIterator.java
+ *
+ * This source file is part of the FoundationDB open source project
+ *
+ * Copyright 2015-2020 Apple Inc. and the FoundationDB project 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.
+ */
+
+package com.apple.foundationdb;
+
+import java.io.Closeable;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Holds the direct buffer that is shared with JNI wrapper. A typical usage is as follows:
+ *
+ * The serialization format of result is =>
+ * [int keyCount, boolean more, ListOf<(int keyLen, int valueLen, byte[] key, byte[] value)>]
+ */
+class RangeResultDirectBufferIterator extends DirectBufferIterator implements Iterator {
+
+ RangeResultDirectBufferIterator(ByteBuffer buffer) { super(buffer); }
+
+ @Override
+ public boolean hasNext() {
+ return super.hasNext();
+ }
+
+ @Override
+ public KeyValue next() {
+ assert (hasResultReady()); // Must be called once its ready.
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+
+ final int keyLen = byteBuffer.getInt();
+ final int valueLen = byteBuffer.getInt();
+ byte[] key = new byte[keyLen];
+ byteBuffer.get(key);
+
+ byte[] value = new byte[valueLen];
+ byteBuffer.get(value);
+
+ current += 1;
+ return new KeyValue(key, value);
+ }
+}
diff --git a/bindings/java/src/main/com/apple/foundationdb/RangeResultSummary.java b/bindings/java/src/main/com/apple/foundationdb/RangeResultSummary.java
index 67d50c18238..5cbfca04af8 100644
--- a/bindings/java/src/main/com/apple/foundationdb/RangeResultSummary.java
+++ b/bindings/java/src/main/com/apple/foundationdb/RangeResultSummary.java
@@ -20,6 +20,8 @@
package com.apple.foundationdb;
+import com.apple.foundationdb.tuple.ByteArrayUtil;
+
class RangeResultSummary {
final byte[] lastKey;
final int keyCount;
@@ -30,4 +32,14 @@ class RangeResultSummary {
this.keyCount = keyCount;
this.more = more;
}
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder("RangeResultSummary{");
+ sb.append("lastKey=").append(ByteArrayUtil.printable(lastKey));
+ sb.append(", keyCount=").append(keyCount);
+ sb.append(", more=").append(more);
+ sb.append('}');
+ return sb.toString();
+ }
}
diff --git a/bindings/java/src/main/com/apple/foundationdb/ReadTransaction.java b/bindings/java/src/main/com/apple/foundationdb/ReadTransaction.java
index b2b81553ef5..417068441de 100644
--- a/bindings/java/src/main/com/apple/foundationdb/ReadTransaction.java
+++ b/bindings/java/src/main/com/apple/foundationdb/ReadTransaction.java
@@ -457,8 +457,8 @@ AsyncIterable getRange(Range range,
*
* @return a handle to access the results of the asynchronous call
*/
- AsyncIterable getRangeAndFlatMap(KeySelector begin, KeySelector end, byte[] mapper, int limit,
- boolean reverse, StreamingMode mode);
+ AsyncIterable getMappedRange(KeySelector begin, KeySelector end, byte[] mapper, int limit,
+ boolean reverse, StreamingMode mode);
/**
* Gets an estimate for the number of bytes stored in the given range.
diff --git a/bindings/java/src/tests.cmake b/bindings/java/src/tests.cmake
index b84c148ac25..40a097da5dd 100644
--- a/bindings/java/src/tests.cmake
+++ b/bindings/java/src/tests.cmake
@@ -52,7 +52,7 @@ set(JAVA_INTEGRATION_TESTS
src/integration/com/apple/foundationdb/CycleMultiClientIntegrationTest.java
src/integration/com/apple/foundationdb/SidebandMultiThreadClientTest.java
src/integration/com/apple/foundationdb/RepeatableReadMultiThreadClientTest.java
- src/integration/com/apple/foundationdb/RangeAndFlatMapQueryIntegrationTest.java
+ src/integration/com/apple/foundationdb/MappedRangeQueryIntegrationTest.java
)
# Resources that are used in integration testing, but are not explicitly test files (JUnit rules,
diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h
index f98aabcb648..f8c8fb58b39 100644
--- a/fdbclient/DatabaseContext.h
+++ b/fdbclient/DatabaseContext.h
@@ -446,7 +446,7 @@ class DatabaseContext : public ReferenceCounted, public FastAll
Counter transactionGetKeyRequests;
Counter transactionGetValueRequests;
Counter transactionGetRangeRequests;
- Counter transactionGetRangeAndFlatMapRequests;
+ Counter transactionGetMappedRangeRequests;
Counter transactionGetRangeStreamRequests;
Counter transactionWatchRequests;
Counter transactionGetAddressesForKeyRequests;
diff --git a/fdbclient/FDBTypes.h b/fdbclient/FDBTypes.h
index 48a8d6d3e35..f405fa7d131 100644
--- a/fdbclient/FDBTypes.h
+++ b/fdbclient/FDBTypes.h
@@ -475,6 +475,7 @@ using KeyRange = Standalone;
using KeyValue = Standalone;
using KeySelector = Standalone;
using RangeResult = Standalone;
+using MappedRangeResult = Standalone;
enum { invalidVersion = -1, latestVersion = -2, MAX_VERSION = std::numeric_limits::max() };
@@ -616,6 +617,8 @@ KeyRangeWith keyRangeWith(const KeyRangeRef& range, const Val& value) {
return KeyRangeWith(range, value);
}
+struct MappedKeyValueRef;
+
struct GetRangeLimits {
enum { ROW_LIMIT_UNLIMITED = -1, BYTE_LIMIT_UNLIMITED = -1 };
@@ -629,6 +632,8 @@ struct GetRangeLimits {
void decrement(VectorRef const& data);
void decrement(KeyValueRef const& data);
+ void decrement(VectorRef const& data);
+ void decrement(MappedKeyValueRef const& data);
// True if either the row or byte limit has been reached
bool isReached();
@@ -689,6 +694,114 @@ struct Traceable : std::true_type {
}
};
+// Similar to KeyValueRef, but result can be empty.
+struct GetValueReqAndResultRef {
+ KeyRef key;
+ Optional result;
+
+ GetValueReqAndResultRef() {}
+ GetValueReqAndResultRef(Arena& a, const GetValueReqAndResultRef& copyFrom)
+ : key(a, copyFrom.key), result(a, copyFrom.result) {}
+
+ bool operator==(const GetValueReqAndResultRef& rhs) const { return key == rhs.key && result == rhs.result; }
+ bool operator!=(const GetValueReqAndResultRef& rhs) const { return !(rhs == *this); }
+ int expectedSize() const { return key.expectedSize() + result.expectedSize(); }
+
+ template
+ void serialize(Ar& ar) {
+ serializer(ar, key, result);
+ }
+};
+
+struct GetRangeReqAndResultRef {
+ KeySelectorRef begin, end;
+ RangeResultRef result;
+
+ GetRangeReqAndResultRef() {}
+ // KeyValueRef(const KeyRef& key, const ValueRef& value) : key(key), value(value) {}
+ GetRangeReqAndResultRef(Arena& a, const GetRangeReqAndResultRef& copyFrom)
+ : begin(a, copyFrom.begin), end(a, copyFrom.end), result(a, copyFrom.result) {}
+
+ bool operator==(const GetRangeReqAndResultRef& rhs) const {
+ return begin == rhs.begin && end == rhs.end && result == rhs.result;
+ }
+ bool operator!=(const GetRangeReqAndResultRef& rhs) const { return !(rhs == *this); }
+
+ template
+ void serialize(Ar& ar) {
+ serializer(ar, begin, end, result);
+ }
+};
+
+using MappedReqAndResultRef = std::variant;
+
+struct MappedKeyValueRef : KeyValueRef {
+ // Save the original key value at the base (KeyValueRef).
+
+ MappedReqAndResultRef reqAndResult;
+
+ MappedKeyValueRef() = default;
+ MappedKeyValueRef(Arena& a, const MappedKeyValueRef& copyFrom) : KeyValueRef(a, copyFrom) {
+ const auto& reqAndResultCopyFrom = copyFrom.reqAndResult;
+ if (std::holds_alternative(reqAndResultCopyFrom)) {
+ auto getValue = std::get(reqAndResultCopyFrom);
+ reqAndResult = GetValueReqAndResultRef(a, getValue);
+ } else if (std::holds_alternative(reqAndResultCopyFrom)) {
+ auto getRange = std::get(reqAndResultCopyFrom);
+ reqAndResult = GetRangeReqAndResultRef(a, getRange);
+ } else {
+ throw internal_error();
+ }
+ }
+
+ bool operator==(const MappedKeyValueRef& rhs) const {
+ return static_cast(*this) == static_cast(rhs) &&
+ reqAndResult == rhs.reqAndResult;
+ }
+ bool operator!=(const MappedKeyValueRef& rhs) const { return !(rhs == *this); }
+
+ // It relies on the base to provide the expectedSize. TODO: Consider add the underlying request and key values into
+ // expected size?
+ // int expectedSize() const { return ((KeyValueRef*)this)->expectedSisze() + reqA }
+
+ template
+ void serialize(Ar& ar) {
+ serializer(ar, ((KeyValueRef&)*this), reqAndResult);
+ }
+};
+
+struct MappedRangeResultRef : VectorRef {
+ // Additional information on range result. See comments on RangeResultRef.
+ bool more;
+ Optional readThrough;
+ bool readToBegin;
+ bool readThroughEnd;
+
+ MappedRangeResultRef() : more(false), readToBegin(false), readThroughEnd(false) {}
+ MappedRangeResultRef(Arena& p, const MappedRangeResultRef& toCopy)
+ : VectorRef(p, toCopy), more(toCopy.more),
+ readThrough(toCopy.readThrough.present() ? KeyRef(p, toCopy.readThrough.get()) : Optional()),
+ readToBegin(toCopy.readToBegin), readThroughEnd(toCopy.readThroughEnd) {}
+ MappedRangeResultRef(const VectorRef& value,
+ bool more,
+ Optional readThrough = Optional())
+ : VectorRef(value), more(more), readThrough(readThrough), readToBegin(false),
+ readThroughEnd(false) {}
+ MappedRangeResultRef(bool readToBegin, bool readThroughEnd)
+ : more(false), readToBegin(readToBegin), readThroughEnd(readThroughEnd) {}
+
+ template
+ void serialize(Ar& ar) {
+ serializer(ar, ((VectorRef&)*this), more, readThrough, readToBegin, readThroughEnd);
+ }
+
+ std::string toString() const {
+ return "more:" + std::to_string(more) +
+ " readThrough:" + (readThrough.present() ? readThrough.get().toString() : "[unset]") +
+ " readToBegin:" + std::to_string(readToBegin) + " readThroughEnd:" + std::to_string(readThroughEnd);
+ }
+};
+
struct KeyValueStoreType {
constexpr static FileIdentifier file_identifier = 6560359;
// These enumerated values are stored in the database configuration, so should NEVER be changed.
diff --git a/fdbclient/IClientApi.h b/fdbclient/IClientApi.h
index f62d588591f..34ab01b445d 100644
--- a/fdbclient/IClientApi.h
+++ b/fdbclient/IClientApi.h
@@ -59,12 +59,12 @@ class ITransaction {
GetRangeLimits limits,
bool snapshot = false,
bool reverse = false) = 0;
- virtual ThreadFuture getRangeAndFlatMap(const KeySelectorRef& begin,
- const KeySelectorRef& end,
- const StringRef& mapper,
- GetRangeLimits limits,
- bool snapshot = false,
- bool reverse = false) = 0;
+ virtual ThreadFuture getMappedRange(const KeySelectorRef& begin,
+ const KeySelectorRef& end,
+ const StringRef& mapper,
+ GetRangeLimits limits,
+ bool snapshot = false,
+ bool reverse = false) = 0;
virtual ThreadFuture>> getAddressesForKey(const KeyRef& key) = 0;
virtual ThreadFuture> getVersionstamp() = 0;
diff --git a/fdbclient/ISingleThreadTransaction.h b/fdbclient/ISingleThreadTransaction.h
index d575bb221a1..23448e45790 100644
--- a/fdbclient/ISingleThreadTransaction.h
+++ b/fdbclient/ISingleThreadTransaction.h
@@ -63,12 +63,12 @@ class ISingleThreadTransaction : public ReferenceCounted getRangeAndFlatMap(KeySelector begin,
- KeySelector end,
- Key mapper,
- GetRangeLimits limits,
- Snapshot = Snapshot::False,
- Reverse = Reverse::False) = 0;
+ virtual Future getMappedRange(KeySelector begin,
+ KeySelector end,
+ Key mapper,
+ GetRangeLimits limits,
+ Snapshot = Snapshot::False,
+ Reverse = Reverse::False) = 0;
virtual Future>> getAddressesForKey(Key const& key) = 0;
virtual Future>> getRangeSplitPoints(KeyRange const& range, int64_t chunkSize) = 0;
virtual Future getEstimatedRangeSizeBytes(KeyRange const& keys) = 0;
diff --git a/fdbclient/MultiVersionTransaction.actor.cpp b/fdbclient/MultiVersionTransaction.actor.cpp
index 39dac56bf94..6719a43d7d5 100644
--- a/fdbclient/MultiVersionTransaction.actor.cpp
+++ b/fdbclient/MultiVersionTransaction.actor.cpp
@@ -146,38 +146,39 @@ ThreadFuture DLTransaction::getRange(const KeyRangeRef& keys,
return getRange(firstGreaterOrEqual(keys.begin), firstGreaterOrEqual(keys.end), limits, snapshot, reverse);
}
-ThreadFuture DLTransaction::getRangeAndFlatMap(const KeySelectorRef& begin,
- const KeySelectorRef& end,
- const StringRef& mapper,
- GetRangeLimits limits,
- bool snapshot,
- bool reverse) {
- FdbCApi::FDBFuture* f = api->transactionGetRangeAndFlatMap(tr,
- begin.getKey().begin(),
- begin.getKey().size(),
- begin.orEqual,
- begin.offset,
- end.getKey().begin(),
- end.getKey().size(),
- end.orEqual,
- end.offset,
- mapper.begin(),
- mapper.size(),
- limits.rows,
- limits.bytes,
- FDB_STREAMING_MODE_EXACT,
- 0,
- snapshot,
- reverse);
- return toThreadFuture(api, f, [](FdbCApi::FDBFuture* f, FdbCApi* api) {
- const FdbCApi::FDBKeyValue* kvs;
+ThreadFuture DLTransaction::getMappedRange(const KeySelectorRef& begin,
+ const KeySelectorRef& end,
+ const StringRef& mapper,
+ GetRangeLimits limits,
+ bool snapshot,
+ bool reverse) {
+ FdbCApi::FDBFuture* f = api->transactionGetMappedRange(tr,
+ begin.getKey().begin(),
+ begin.getKey().size(),
+ begin.orEqual,
+ begin.offset,
+ end.getKey().begin(),
+ end.getKey().size(),
+ end.orEqual,
+ end.offset,
+ mapper.begin(),
+ mapper.size(),
+ limits.rows,
+ limits.bytes,
+ FDB_STREAMING_MODE_EXACT,
+ 0,
+ snapshot,
+ reverse);
+ return toThreadFuture(api, f, [](FdbCApi::FDBFuture* f, FdbCApi* api) {
+ const FdbCApi::FDBMappedKeyValue* kvms;
int count;
FdbCApi::fdb_bool_t more;
- FdbCApi::fdb_error_t error = api->futureGetKeyValueArray(f, &kvs, &count, &more);
+ FdbCApi::fdb_error_t error = api->futureGetMappedKeyValueArray(f, &kvms, &count, &more);
ASSERT(!error);
// The memory for this is stored in the FDBFuture and is released when the future gets destroyed
- return RangeResult(RangeResultRef(VectorRef((KeyValueRef*)kvs, count), more), Arena());
+ return MappedRangeResult(
+ MappedRangeResultRef(VectorRef((MappedKeyValueRef*)kvms, count), more), Arena());
});
}
@@ -555,11 +556,8 @@ void DLApi::init() {
"fdb_transaction_get_addresses_for_key",
headerVersion >= 0);
loadClientFunction(&api->transactionGetRange, lib, fdbCPath, "fdb_transaction_get_range", headerVersion >= 0);
- loadClientFunction(&api->transactionGetRangeAndFlatMap,
- lib,
- fdbCPath,
- "fdb_transaction_get_range_and_flat_map",
- headerVersion >= 700);
+ loadClientFunction(
+ &api->transactionGetMappedRange, lib, fdbCPath, "fdb_transaction_get_mapped_range", headerVersion >= 700);
loadClientFunction(
&api->transactionGetVersionstamp, lib, fdbCPath, "fdb_transaction_get_versionstamp", headerVersion >= 410);
loadClientFunction(&api->transactionSet, lib, fdbCPath, "fdb_transaction_set", headerVersion >= 0);
@@ -616,6 +614,8 @@ void DLApi::init() {
loadClientFunction(&api->futureGetKeyArray, lib, fdbCPath, "fdb_future_get_key_array", headerVersion >= 700);
loadClientFunction(
&api->futureGetKeyValueArray, lib, fdbCPath, "fdb_future_get_keyvalue_array", headerVersion >= 0);
+ loadClientFunction(
+ &api->futureGetMappedKeyValueArray, lib, fdbCPath, "fdb_future_get_mappedkeyvalue_array", headerVersion >= 700);
loadClientFunction(&api->futureSetCallback, lib, fdbCPath, "fdb_future_set_callback", headerVersion >= 0);
loadClientFunction(&api->futureCancel, lib, fdbCPath, "fdb_future_cancel", headerVersion >= 0);
loadClientFunction(&api->futureDestroy, lib, fdbCPath, "fdb_future_destroy", headerVersion >= 0);
@@ -861,15 +861,15 @@ ThreadFuture MultiVersionTransaction::getRange(const KeyRangeRef& k
return abortableFuture(f, tr.onChange);
}
-ThreadFuture MultiVersionTransaction::getRangeAndFlatMap(const KeySelectorRef& begin,
- const KeySelectorRef& end,
- const StringRef& mapper,
- GetRangeLimits limits,
- bool snapshot,
- bool reverse) {
+ThreadFuture MultiVersionTransaction::getMappedRange(const KeySelectorRef& begin,
+ const KeySelectorRef& end,
+ const StringRef& mapper,
+ GetRangeLimits limits,
+ bool snapshot,
+ bool reverse) {
auto tr = getTransaction();
- auto f = tr.transaction ? tr.transaction->getRangeAndFlatMap(begin, end, mapper, limits, snapshot, reverse)
- : makeTimeout();
+ auto f = tr.transaction ? tr.transaction->getMappedRange(begin, end, mapper, limits, snapshot, reverse)
+ : makeTimeout();
return abortableFuture(f, tr.onChange);
}
diff --git a/fdbclient/MultiVersionTransaction.h b/fdbclient/MultiVersionTransaction.h
index f8bde8c000d..87556b4c679 100644
--- a/fdbclient/MultiVersionTransaction.h
+++ b/fdbclient/MultiVersionTransaction.h
@@ -38,6 +38,9 @@ struct FdbCApi : public ThreadSafeReferenceCounted {
typedef struct FDB_database FDBDatabase;
typedef struct FDB_transaction FDBTransaction;
+ typedef int fdb_error_t;
+ typedef int fdb_bool_t;
+
#pragma pack(push, 4)
typedef struct key {
const uint8_t* key;
@@ -49,6 +52,35 @@ struct FdbCApi : public ThreadSafeReferenceCounted {
const void* value;
int valueLength;
} FDBKeyValue;
+
+#pragma pack(pop)
+
+ /* Memory layout of KeySelectorRef. */
+ typedef struct keyselector {
+ FDBKey key;
+ /* orEqual and offset have not be tested in C binding. Just a placeholder. */
+ fdb_bool_t orEqual;
+ int offset;
+ } FDBKeySelector;
+
+ /* Memory layout of GetRangeReqAndResultRef. */
+ typedef struct getrangereqandresult {
+ FDBKeySelector begin;
+ FDBKeySelector end;
+ FDBKeyValue* data;
+ int m_size, m_capacity;
+ } FDBGetRangeReqAndResult;
+
+ typedef struct mappedkeyvalue {
+ FDBKey key;
+ FDBKey value;
+ /* It's complicated to map a std::variant to C. For now we assume the underlying requests are always getRange
+ * and take the shortcut. */
+ FDBGetRangeReqAndResult getRange;
+ unsigned char buffer[32];
+ } FDBMappedKeyValue;
+
+#pragma pack(push, 4)
typedef struct keyrange {
const void* beginKey;
int beginKeyLength;
@@ -57,9 +89,6 @@ struct FdbCApi : public ThreadSafeReferenceCounted {
} FDBKeyRange;
#pragma pack(pop)
- typedef int fdb_error_t;
- typedef int fdb_bool_t;
-
typedef struct readgranulecontext {
// User context to pass along to functions
void* userContext;
@@ -144,23 +173,23 @@ struct FdbCApi : public ThreadSafeReferenceCounted {
int iteration,
fdb_bool_t snapshot,
fdb_bool_t reverse);
- FDBFuture* (*transactionGetRangeAndFlatMap)(FDBTransaction* tr,
- uint8_t const* beginKeyName,
- int beginKeyNameLength,
- fdb_bool_t beginOrEqual,
- int beginOffset,
- uint8_t const* endKeyName,
- int endKeyNameLength,
- fdb_bool_t endOrEqual,
- int endOffset,
- uint8_t const* mapper_name,
- int mapper_name_length,
- int limit,
- int targetBytes,
- FDBStreamingMode mode,
- int iteration,
- fdb_bool_t snapshot,
- fdb_bool_t reverse);
+ FDBFuture* (*transactionGetMappedRange)(FDBTransaction* tr,
+ uint8_t const* beginKeyName,
+ int beginKeyNameLength,
+ fdb_bool_t beginOrEqual,
+ int beginOffset,
+ uint8_t const* endKeyName,
+ int endKeyNameLength,
+ fdb_bool_t endOrEqual,
+ int endOffset,
+ uint8_t const* mapper_name,
+ int mapper_name_length,
+ int limit,
+ int targetBytes,
+ FDBStreamingMode mode,
+ int iteration,
+ fdb_bool_t snapshot,
+ fdb_bool_t reverse);
FDBFuture* (*transactionGetVersionstamp)(FDBTransaction* tr);
void (*transactionSet)(FDBTransaction* tr,
@@ -236,6 +265,10 @@ struct FdbCApi : public ThreadSafeReferenceCounted {
fdb_error_t (*futureGetKeyRangeArray)(FDBFuture* f, const FDBKeyRange** out_keyranges, int* outCount);
fdb_error_t (*futureGetKeyArray)(FDBFuture* f, FDBKey const** outKeys, int* outCount);
fdb_error_t (*futureGetKeyValueArray)(FDBFuture* f, FDBKeyValue const** outKV, int* outCount, fdb_bool_t* outMore);
+ fdb_error_t (*futureGetMappedKeyValueArray)(FDBFuture* f,
+ FDBMappedKeyValue const** outKVM,
+ int* outCount,
+ fdb_bool_t* outMore);
fdb_error_t (*futureSetCallback)(FDBFuture* f, FDBCallback callback, void* callback_parameter);
void (*futureCancel)(FDBFuture* f);
void (*futureDestroy)(FDBFuture* f);
@@ -281,12 +314,12 @@ class DLTransaction : public ITransaction, ThreadSafeReferenceCounted getRangeAndFlatMap(const KeySelectorRef& begin,
- const KeySelectorRef& end,
- const StringRef& mapper,
- GetRangeLimits limits,
- bool snapshot,
- bool reverse) override;
+ ThreadFuture getMappedRange(const KeySelectorRef& begin,
+ const KeySelectorRef& end,
+ const StringRef& mapper,
+ GetRangeLimits limits,
+ bool snapshot,
+ bool reverse) override;
ThreadFuture>> getAddressesForKey(const KeyRef& key) override;
ThreadFuture> getVersionstamp() override;
ThreadFuture getEstimatedRangeSizeBytes(const KeyRangeRef& keys) override;
@@ -434,12 +467,12 @@ class MultiVersionTransaction : public ITransaction, ThreadSafeReferenceCounted<
GetRangeLimits limits,
bool snapshot = false,
bool reverse = false) override;
- ThreadFuture getRangeAndFlatMap(const KeySelectorRef& begin,
- const KeySelectorRef& end,
- const StringRef& mapper,
- GetRangeLimits limits,
- bool snapshot,
- bool reverse) override;
+ ThreadFuture getMappedRange(const KeySelectorRef& begin,
+ const KeySelectorRef& end,
+ const StringRef& mapper,
+ GetRangeLimits limits,
+ bool snapshot,
+ bool reverse) override;
ThreadFuture>> getAddressesForKey(const KeyRef& key) override;
ThreadFuture> getVersionstamp() override;
diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp
index 8d094401a40..1d4c8989255 100644
--- a/fdbclient/NativeAPI.actor.cpp
+++ b/fdbclient/NativeAPI.actor.cpp
@@ -171,8 +171,8 @@ void DatabaseContext::addTssMapping(StorageServerInterface const& ssi, StorageSe
TSSEndpointData(tssi.id(), tssi.getKey.getEndpoint(), metrics));
queueModel.updateTssEndpoint(ssi.getKeyValues.getEndpoint().token.first(),
TSSEndpointData(tssi.id(), tssi.getKeyValues.getEndpoint(), metrics));
- queueModel.updateTssEndpoint(ssi.getKeyValuesAndFlatMap.getEndpoint().token.first(),
- TSSEndpointData(tssi.id(), tssi.getKeyValuesAndFlatMap.getEndpoint(), metrics));
+ queueModel.updateTssEndpoint(ssi.getMappedKeyValues.getEndpoint().token.first(),
+ TSSEndpointData(tssi.id(), tssi.getMappedKeyValues.getEndpoint(), metrics));
queueModel.updateTssEndpoint(ssi.getKeyValuesStream.getEndpoint().token.first(),
TSSEndpointData(tssi.id(), tssi.getKeyValuesStream.getEndpoint(), metrics));
@@ -196,7 +196,7 @@ void DatabaseContext::removeTssMapping(StorageServerInterface const& ssi) {
queueModel.removeTssEndpoint(ssi.getValue.getEndpoint().token.first());
queueModel.removeTssEndpoint(ssi.getKey.getEndpoint().token.first());
queueModel.removeTssEndpoint(ssi.getKeyValues.getEndpoint().token.first());
- queueModel.removeTssEndpoint(ssi.getKeyValuesAndFlatMap.getEndpoint().token.first());
+ queueModel.removeTssEndpoint(ssi.getMappedKeyValues.getEndpoint().token.first());
queueModel.removeTssEndpoint(ssi.getKeyValuesStream.getEndpoint().token.first());
queueModel.removeTssEndpoint(ssi.watchValue.getEndpoint().token.first());
@@ -476,9 +476,9 @@ ACTOR Future tssLogger(DatabaseContext* cx) {
tssEv, "GetKeyValuesLatency", it.second->SSgetKeyValuesLatency, it.second->TSSgetKeyValuesLatency);
traceTSSPercentiles(tssEv, "GetKeyLatency", it.second->SSgetKeyLatency, it.second->TSSgetKeyLatency);
traceTSSPercentiles(tssEv,
- "GetKeyValuesAndFlatMapLatency",
- it.second->SSgetKeyValuesAndFlatMapLatency,
- it.second->TSSgetKeyValuesAndFlatMapLatency);
+ "GetMappedKeyValuesLatency",
+ it.second->SSgetMappedKeyValuesLatency,
+ it.second->TSSgetMappedKeyValuesLatency);
it.second->clear();
}
@@ -1314,7 +1314,7 @@ DatabaseContext::DatabaseContext(Reference const& data) {
+ if (rows != GetRangeLimits::ROW_LIMIT_UNLIMITED) {
+ ASSERT(data.size() <= rows);
+ rows -= data.size();
+ }
+
+ minRows = std::max(0, minRows - data.size());
+
+ // TODO: For now, expectedSize only considers the size of the original key values, but not the underlying queries or
+ // results. Also, double check it is correct when dealing with sizeof(MappedKeyValueRef).
+ if (bytes != GetRangeLimits::BYTE_LIMIT_UNLIMITED)
+ bytes = std::max(0, bytes - (int)data.expectedSize() - (8 - (int)sizeof(MappedKeyValueRef)) * data.size());
+}
+
+void GetRangeLimits::decrement(MappedKeyValueRef const& data) {
+ minRows = std::max(0, minRows - 1);
+ if (rows != GetRangeLimits::ROW_LIMIT_UNLIMITED)
+ rows--;
+ // TODO: For now, expectedSize only considers the size of the original key values, but not the underlying queries or
+ // results. Also, double check it is correct when dealing with sizeof(MappedKeyValueRef).
+ if (bytes != GetRangeLimits::BYTE_LIMIT_UNLIMITED)
+ bytes = std::max(0, bytes - (int)8 - (int)data.expectedSize());
+}
+
// True if either the row or byte limit has been reached
bool GetRangeLimits::isReached() {
return rows == 0 || (bytes == 0 && minRows == 0);
@@ -3355,21 +3379,21 @@ template
RequestStream StorageServerInterface::*getRangeRequestStream() {
if constexpr (std::is_same::value) {
return &StorageServerInterface::getKeyValues;
- } else if (std::is_same::value) {
- return &StorageServerInterface::getKeyValuesAndFlatMap;
+ } else if (std::is_same::value) {
+ return &StorageServerInterface::getMappedKeyValues;
} else {
UNREACHABLE();
}
}
-ACTOR template
-Future getExactRange(Reference trState,
- Version version,
- KeyRange keys,
- Key mapper,
- GetRangeLimits limits,
- Reverse reverse) {
- state RangeResult output;
+ACTOR template
+Future getExactRange(Reference trState,
+ Version version,
+ KeyRange keys,
+ Key mapper,
+ GetRangeLimits limits,
+ Reverse reverse) {
+ state RangeResultFamily output;
state Span span("NAPI:getExactRange"_loc, trState->spanID);
// printf("getExactRange( '%s', '%s' )\n", keys.begin.toString().c_str(), keys.end.toString().c_str());
@@ -3547,14 +3571,14 @@ Future resolveKey(Reference trState, KeySelector const& k
return getKey(trState, key, version);
}
-ACTOR template
-Future getRangeFallback(Reference trState,
- Version version,
- KeySelector begin,
- KeySelector end,
- Key mapper,
- GetRangeLimits limits,
- Reverse reverse) {
+ACTOR template
+Future getRangeFallback(Reference trState,
+ Version version,
+ KeySelector begin,
+ KeySelector end,
+ Key mapper,
+ GetRangeLimits limits,
+ Reverse reverse) {
if (version == latestVersion) {
state Transaction transaction(trState->cx);
transaction.setOption(FDBTransactionOptions::CAUSAL_READ_RISKY);
@@ -3570,16 +3594,16 @@ Future getRangeFallback(Reference trState,
state Key b = wait(fb);
state Key e = wait(fe);
if (b >= e) {
- return RangeResult();
+ return RangeResultFamily();
}
// if e is allKeys.end, we have read through the end of the database
// if b is allKeys.begin, we have either read through the beginning of the database,
// or allKeys.begin exists in the database and will be part of the conflict range anyways
- RangeResult _r = wait(getExactRange(
+ RangeResultFamily _r = wait(getExactRange(
trState, version, KeyRangeRef(b, e), mapper, limits, reverse));
- RangeResult r = _r;
+ RangeResultFamily r = _r;
if (b == allKeys.begin && ((reverse && !r.more) || !reverse))
r.readToBegin = true;
@@ -3603,7 +3627,31 @@ Future getRangeFallback(Reference trState,
return r;
}
+int64_t inline getRangeResultFamilyBytes(RangeResultRef result) {
+ return result.expectedSize();
+}
+
+int64_t inline getRangeResultFamilyBytes(MappedRangeResultRef result) {
+ int64_t bytes = 0;
+ for (const MappedKeyValueRef& mappedKeyValue : result) {
+ bytes += mappedKeyValue.key.size() + mappedKeyValue.value.size();
+
+ auto& reqAndResult = mappedKeyValue.reqAndResult;
+ if (std::holds_alternative(reqAndResult)) {
+ auto getValue = std::get(reqAndResult);
+ bytes += getValue.expectedSize();
+ } else if (std::holds_alternative(reqAndResult)) {
+ auto getRange = std::get(reqAndResult);
+ bytes += getRange.result.expectedSize();
+ } else {
+ throw internal_error();
+ }
+ }
+ return bytes;
+}
+
// TODO: Client should add mapped keys to conflict ranges.
+ACTOR template // RangeResult or MappedRangeResult
void getRangeFinished(Reference trState,
double startTime,
KeySelector begin,
@@ -3611,11 +3659,8 @@ void getRangeFinished(Reference trState,
Snapshot snapshot,
Promise> conflictRange,
Reverse reverse,
- RangeResult result) {
- int64_t bytes = 0;
- for (const KeyValueRef& kv : result) {
- bytes += kv.key.size() + kv.value.size();
- }
+ RangeResultFamily result) {
+ int64_t bytes = getRangeResultFamilyBytes(result);
trState->cx->transactionBytesRead += bytes;
trState->cx->transactionKeysRead += result.size();
@@ -3657,24 +3702,26 @@ void getRangeFinished(Reference trState,
}
}
-// GetKeyValuesFamilyRequest: GetKeyValuesRequest or GetKeyValuesAndFlatMapRequest
-// GetKeyValuesFamilyReply: GetKeyValuesReply or GetKeyValuesAndFlatMapReply
-// Sadly we need GetKeyValuesFamilyReply because cannot do something like: state
-// REPLY_TYPE(GetKeyValuesFamilyRequest) rep;
-ACTOR template
-Future getRange(Reference trState,
- Future fVersion,
- KeySelector begin,
- KeySelector end,
- Key mapper,
- GetRangeLimits limits,
- Promise> conflictRange,
- Snapshot snapshot,
- Reverse reverse) {
+ACTOR template
+Future getRange(Reference trState,
+ Future fVersion,
+ KeySelector begin,
+ KeySelector end,
+ Key mapper,
+ GetRangeLimits limits,
+ Promise> conflictRange,
+ Snapshot snapshot,
+ Reverse reverse) {
+ // state using RangeResultRefFamily = typename RangeResultFamily::RefType;
state GetRangeLimits originalLimits(limits);
state KeySelector originalBegin = begin;
state KeySelector originalEnd = end;
- state RangeResult output;
+ state RangeResultFamily output;
state Span span("NAPI:getRange"_loc, trState->spanID);
try {
@@ -3822,15 +3869,16 @@ Future getRange(Reference trState,
bool readToBegin = output.readToBegin;
bool readThroughEnd = output.readThroughEnd;
- output = RangeResult(RangeResultRef(rep.data, modifiedSelectors || limits.isReached() || rep.more),
- rep.arena);
+ using RangeResultRefFamily = typename RangeResultFamily::RefType;
+ output = RangeResultFamily(
+ RangeResultRefFamily(rep.data, modifiedSelectors || limits.isReached() || rep.more), rep.arena);
output.readToBegin = readToBegin;
output.readThroughEnd = readThroughEnd;
if (BUGGIFY && limits.hasByteLimit() && output.size() > std::max(1, originalLimits.minRows)) {
// Copy instead of resizing because TSS maybe be using output's arena for comparison. This only
// happens in simulation so it's fine
- RangeResult copy;
+ RangeResultFamily copy;
int newSize =
deterministicRandom()->randomInt(std::max(1, originalLimits.minRows), output.size());
for (int i = 0; i < newSize; i++) {
@@ -3876,8 +3924,9 @@ Future getRange(Reference trState,
TEST(true); // !GetKeyValuesFamilyReply.more and modifiedSelectors in getRange
if (!rep.data.size()) {
- RangeResult result = wait(getRangeFallback(
- trState, version, originalBegin, originalEnd, mapper, originalLimits, reverse));
+ RangeResultFamily result = wait(
+ getRangeFallback(
+ trState, version, originalBegin, originalEnd, mapper, originalLimits, reverse));
getRangeFinished(
trState, startTime, originalBegin, originalEnd, snapshot, conflictRange, reverse, result);
return result;
@@ -3907,8 +3956,9 @@ Future getRange(Reference trState,
Reverse{ reverse ? (end - 1).isBackward() : begin.isBackward() });
if (e.code() == error_code_wrong_shard_server) {
- RangeResult result = wait(getRangeFallback(
- trState, version, originalBegin, originalEnd, mapper, originalLimits, reverse));
+ RangeResultFamily result = wait(
+ getRangeFallback(
+ trState, version, originalBegin, originalEnd, mapper, originalLimits, reverse));
getRangeFinished(
trState, startTime, originalBegin, originalEnd, snapshot, conflictRange, reverse, result);
return result;
@@ -4461,7 +4511,7 @@ Future getRange(Reference const& trState,
KeySelector const& end,
GetRangeLimits const& limits,
Reverse const& reverse) {
- return getRange(
+ return getRange(
trState, fVersion, begin, end, ""_sr, limits, Promise>(), Snapshot::True, reverse);
}
@@ -4755,25 +4805,25 @@ template
void increaseCounterForRequest(Database cx) {
if constexpr (std::is_same::value) {
++cx->transactionGetRangeRequests;
- } else if (std::is_same::value) {
- ++cx->transactionGetRangeAndFlatMapRequests;
+ } else if (std::is_same::value) {
+ ++cx->transactionGetMappedRangeRequests;
} else {
UNREACHABLE();
}
}
-template
-Future Transaction::getRangeInternal(const KeySelector& begin,
- const KeySelector& end,
- const Key& mapper,
- GetRangeLimits limits,
- Snapshot snapshot,
- Reverse reverse) {
+template
+Future Transaction::getRangeInternal(const KeySelector& begin,
+ const KeySelector& end,
+ const Key& mapper,
+ GetRangeLimits limits,
+ Snapshot snapshot,
+ Reverse reverse) {
++trState->cx->transactionLogicalReads;
increaseCounterForRequest(trState->cx);
if (limits.isReached())
- return RangeResult();
+ return RangeResultFamily();
if (!limits.isValid())
return range_limits_invalid();
@@ -4794,15 +4844,21 @@ Future Transaction::getRangeInternal(const KeySelector& begin,
if (b.offset >= e.offset && b.getKey() >= e.getKey()) {
TEST(true); // Native range inverted
- return RangeResult();
+ return RangeResultFamily();
}
+ if (!snapshot && !std::is_same_v) {
+ // Currently, NativeAPI does not support serialization for getMappedRange. You should consider use
+ // ReadYourWrites APIs which wraps around NativeAPI and provides serialization for getMappedRange. (Even if
+ // you don't want RYW, you may use ReadYourWrites APIs with RYW disabled.)
+ throw unsupported_operation();
+ }
Promise> conflictRange;
if (!snapshot) {
extraConflictRanges.push_back(conflictRange.getFuture());
}
- return ::getRange(
+ return ::getRange(
trState, getReadVersion(), b, e, mapper, limits, conflictRange, snapshot, reverse);
}
@@ -4811,16 +4867,17 @@ Future Transaction::getRange(const KeySelector& begin,
GetRangeLimits limits,
Snapshot snapshot,
Reverse reverse) {
- return getRangeInternal(begin, end, ""_sr, limits, snapshot, reverse);
+ return getRangeInternal(
+ begin, end, ""_sr, limits, snapshot, reverse);
}
-Future Transaction::getRangeAndFlatMap(const KeySelector& begin,
- const KeySelector& end,
- const Key& mapper,
- GetRangeLimits limits,
- Snapshot snapshot,
- Reverse reverse) {
- return getRangeInternal(
+Future Transaction::getMappedRange(const KeySelector& begin,
+ const KeySelector& end,
+ const Key& mapper,
+ GetRangeLimits limits,
+ Snapshot snapshot,
+ Reverse reverse) {
+ return getRangeInternal(
begin, end, mapper, limits, snapshot, reverse);
}
diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h
index 29e2dda610e..d2ee31d20b1 100644
--- a/fdbclient/NativeAPI.actor.h
+++ b/fdbclient/NativeAPI.actor.h
@@ -309,13 +309,23 @@ class Transaction : NonCopyable {
reverse);
}
- [[nodiscard]] Future getRangeAndFlatMap(const KeySelector& begin,
- const KeySelector& end,
- const Key& mapper,
- GetRangeLimits limits,
- Snapshot = Snapshot::False,
- Reverse = Reverse::False);
+ [[nodiscard]] Future getMappedRange(const KeySelector& begin,
+ const KeySelector& end,
+ const Key& mapper,
+ GetRangeLimits limits,
+ Snapshot = Snapshot::False,
+ Reverse = Reverse::False);
+private:
+ template
+ Future getRangeInternal(const KeySelector& begin,
+ const KeySelector& end,
+ const Key& mapper,
+ GetRangeLimits limits,
+ Snapshot snapshot,
+ Reverse reverse);
+
+public:
// A method for streaming data from the storage server that is more efficient than getRange when reading large
// amounts of data
[[nodiscard]] Future getRangeStream(const PromiseStream>& results,
diff --git a/fdbclient/PaxosConfigTransaction.h b/fdbclient/PaxosConfigTransaction.h
index 3854d4be963..276450c8a42 100644
--- a/fdbclient/PaxosConfigTransaction.h
+++ b/fdbclient/PaxosConfigTransaction.h
@@ -50,12 +50,12 @@ class PaxosConfigTransaction final : public IConfigTransaction, public FastAlloc
GetRangeLimits limits,
Snapshot = Snapshot::False,
Reverse = Reverse::False) override;
- Future getRangeAndFlatMap(KeySelector begin,
- KeySelector end,
- Key mapper,
- GetRangeLimits limits,
- Snapshot = Snapshot::False,
- Reverse = Reverse::False) override {
+ Future getMappedRange(KeySelector begin,
+ KeySelector end,
+ Key mapper,
+ GetRangeLimits limits,
+ Snapshot = Snapshot::False,
+ Reverse = Reverse::False) override {
throw client_invalid_operation();
}
void set(KeyRef const& key, ValueRef const& value) override;
diff --git a/fdbclient/RYWIterator.h b/fdbclient/RYWIterator.h
index 90ab1884e09..fb5b4768bd7 100644
--- a/fdbclient/RYWIterator.h
+++ b/fdbclient/RYWIterator.h
@@ -44,7 +44,7 @@ class RYWIterator {
ExtStringRef beginKey();
ExtStringRef endKey();
- const KeyValueRef* kv(Arena& arena);
+ virtual const KeyValueRef* kv(Arena& arena);
RYWIterator& operator++();
@@ -61,14 +61,14 @@ class RYWIterator {
void bypassUnreadableProtection() { bypassUnreadable = true; }
- WriteMap::iterator& extractWriteMapIterator();
+ virtual WriteMap::iterator& extractWriteMapIterator();
// Really this should return an iterator by value, but for performance it's convenient to actually grab the internal
// one. Consider copying the return value if performance isn't critical. If you modify the returned iterator, it
// invalidates this iterator until the next call to skip()
void dbg();
-private:
+protected:
int begin_key_cmp; // -1 if cache.beginKey() < writes.beginKey(), 0 if ==, +1 if >
int end_key_cmp; //
SnapshotCache::iterator cache;
diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp
index ee021b6a054..bc123024728 100644
--- a/fdbclient/ReadYourWrites.actor.cpp
+++ b/fdbclient/ReadYourWrites.actor.cpp
@@ -75,13 +75,13 @@ class RYWImpl {
};
template
- struct GetRangeAndFlatMapReq {
- GetRangeAndFlatMapReq(KeySelector begin, KeySelector end, Key mapper, GetRangeLimits limits)
+ struct GetMappedRangeReq {
+ GetMappedRangeReq(KeySelector begin, KeySelector end, Key mapper, GetRangeLimits limits)
: begin(begin), end(end), mapper(mapper), limits(limits) {}
KeySelector begin, end;
Key mapper;
GetRangeLimits limits;
- using Result = RangeResult;
+ using Result = MappedRangeResult;
};
// read() Performs a read (get, getKey, getRange, etc), in the context of the given transaction. Snapshot or RYW
@@ -213,46 +213,17 @@ class RYWImpl {
return v;
}
- ACTOR template
- static Future readThroughAndFlatMap(ReadYourWritesTransaction* ryw,
- GetRangeAndFlatMapReq read,
- Snapshot snapshot) {
- if (backwards && read.end.offset > 1) {
- // FIXME: Optimistically assume that this will not run into the system keys, and only reissue if the result
- // actually does.
- Key key = wait(ryw->tr.getKey(read.end, snapshot));
- if (key > ryw->getMaxReadKey())
- read.end = firstGreaterOrEqual(ryw->getMaxReadKey());
- else
- read.end = KeySelector(firstGreaterOrEqual(key), key.arena());
- }
-
- RangeResult v = wait(ryw->tr.getRangeAndFlatMap(
- read.begin, read.end, read.mapper, read.limits, snapshot, backwards ? Reverse::True : Reverse::False));
- KeyRef maxKey = ryw->getMaxReadKey();
- if (v.size() > 0) {
- if (!backwards && v[v.size() - 1].key >= maxKey) {
- state RangeResult _v = v;
- int i = _v.size() - 2;
- for (; i >= 0 && _v[i].key >= maxKey; --i) {
- }
- return RangeResult(RangeResultRef(VectorRef(&_v[0], i + 1), false), _v.arena());
- }
- }
-
- return v;
- }
-
// addConflictRange(ryw,read,result) is called after a serializable read and is responsible for adding the relevant
// conflict range
+ template
static void addConflictRange(ReadYourWritesTransaction* ryw,
GetValueReq read,
WriteMap::iterator& it,
Optional result) {
// it will already point to the right segment (see the calling code in read()), so we don't need to skip
// read.key will be copied into ryw->arena inside of updateConflictMap if it is being added
- ryw->updateConflictMap(read.key, it);
+ updateConflictMap(ryw, read.key, it);
}
static void addConflictRange(ReadYourWritesTransaction* ryw, GetKeyReq read, WriteMap::iterator& it, Key result) {
@@ -270,10 +241,11 @@ class RYWImpl {
ryw->updateConflictMap(readRange, it);
}
+ template
static void addConflictRange(ReadYourWritesTransaction* ryw,
GetRangeReq read,
WriteMap::iterator& it,
- RangeResult const& result) {
+ RangeResultFamily& result) {
KeyRef rangeBegin, rangeEnd;
bool endInArena = false;
@@ -302,13 +274,15 @@ class RYWImpl {
KeyRangeRef readRange =
KeyRangeRef(KeyRef(ryw->arena, rangeBegin), endInArena ? rangeEnd : KeyRef(ryw->arena, rangeEnd));
it.skip(readRange.begin);
- ryw->updateConflictMap(readRange, it);
+ updateConflictMap(ryw, readRange, it);
}
+ // In the case where RangeResultFamily is MappedRangeResult, it only adds the primary range to conflict.
+ template
static void addConflictRange(ReadYourWritesTransaction* ryw,
GetRangeReq read,
WriteMap::iterator& it,
- RangeResult const& result) {
+ RangeResultFamily& result) {
KeyRef rangeBegin, rangeEnd;
bool endInArena = false;
@@ -336,7 +310,39 @@ class RYWImpl {
KeyRangeRef readRange =
KeyRangeRef(KeyRef(ryw->arena, rangeBegin), endInArena ? rangeEnd : KeyRef(ryw->arena, rangeEnd));
it.skip(readRange.begin);
- ryw->updateConflictMap(readRange, it);
+ updateConflictMap(ryw, readRange, it);
+ }
+
+ template
+ static void updateConflictMap(ReadYourWritesTransaction* ryw, KeyRef const& key, WriteMap::iterator& it) {
+ // it.skip( key );
+ // ASSERT( it.beginKey() <= key && key < it.endKey() );
+ if (mustUnmodified && !it.is_unmodified_range()) {
+ throw get_mapped_range_reads_your_writes();
+ }
+ if (it.is_unmodified_range() || (it.is_operation() && !it.is_independent())) {
+ ryw->approximateSize += 2 * key.expectedSize() + 1 + sizeof(KeyRangeRef);
+ ryw->readConflicts.insert(singleKeyRange(key, ryw->arena), true);
+ }
+ }
+
+ template
+ static void updateConflictMap(ReadYourWritesTransaction* ryw, KeyRangeRef const& keys, WriteMap::iterator& it) {
+ // it.skip( keys.begin );
+ // ASSERT( it.beginKey() <= keys.begin && keys.begin < it.endKey() );
+ for (; it.beginKey() < keys.end; ++it) {
+ if (mustUnmodified && !it.is_unmodified_range()) {
+ throw get_mapped_range_reads_your_writes();
+ }
+ if (it.is_unmodified_range() || (it.is_operation() && !it.is_independent())) {
+ KeyRangeRef insert_range = KeyRangeRef(std::max(keys.begin, it.beginKey().toArenaOrRef(ryw->arena)),
+ std::min(keys.end, it.endKey().toArenaOrRef(ryw->arena)));
+ if (!insert_range.empty()) {
+ ryw->approximateSize += keys.expectedSize() + sizeof(KeyRangeRef);
+ ryw->readConflicts.insert(insert_range, true);
+ }
+ }
+ }
}
ACTOR template
@@ -349,15 +355,6 @@ class RYWImpl {
}
}
ACTOR template
- static Future readWithConflictRangeThroughAndFlatMap(ReadYourWritesTransaction* ryw,
- Req req,
- Snapshot snapshot) {
- choose {
- when(typename Req::Result result = wait(readThroughAndFlatMap(ryw, req, snapshot))) { return result; }
- when(wait(ryw->resetPromise.getFuture())) { throw internal_error(); }
- }
- }
- ACTOR template
static Future readWithConflictRangeSnapshot(ReadYourWritesTransaction* ryw, Req req) {
state SnapshotCache::iterator it(&ryw->cache, &ryw->writes);
choose {
@@ -393,19 +390,6 @@ class RYWImpl {
return readWithConflictRangeRYW(ryw, req, snapshot);
}
- template
- static inline Future readWithConflictRangeAndFlatMap(ReadYourWritesTransaction* ryw,
- Req const& req,
- Snapshot snapshot) {
- // For now, getRangeAndFlatMap is only supported if transaction use snapshot isolation AND read-your-writes is
- // disabled.
- if (snapshot && ryw->options.readYourWritesDisabled) {
- return readWithConflictRangeThroughAndFlatMap(ryw, req, snapshot);
- }
- TEST(true); // readWithConflictRangeRYW not supported for getRangeAndFlatMap
- throw client_invalid_operation();
- }
-
template
static void resolveKeySelectorFromCache(KeySelector& key,
Iter& it,
@@ -1126,6 +1110,119 @@ class RYWImpl {
return result;
}
+#ifndef __INTEL_COMPILER
+#pragma region GetMappedRange
+#endif
+
+ template
+ static Future read(ReadYourWritesTransaction* ryw, GetMappedRangeReq read, Iter* it) {
+ return getMappedRangeValue(ryw, read.begin, read.end, read.mapper, read.limits, it);
+ };
+
+ template
+ static Future read(ReadYourWritesTransaction* ryw, GetMappedRangeReq read, Iter* it) {
+ throw unsupported_operation();
+ // TODO: Support reverse. return getMappedRangeValueBack(ryw, read.begin, read.end, read.mapper,
+ // read.limits, it);
+ };
+
+ ACTOR template
+ static Future readThrough(ReadYourWritesTransaction* ryw,
+ GetMappedRangeReq read,
+ Snapshot snapshot) {
+ if (backwards && read.end.offset > 1) {
+ // FIXME: Optimistically assume that this will not run into the system keys, and only reissue if the result
+ // actually does.
+ Key key = wait(ryw->tr.getKey(read.end, snapshot));
+ if (key > ryw->getMaxReadKey())
+ read.end = firstGreaterOrEqual(ryw->getMaxReadKey());
+ else
+ read.end = KeySelector(firstGreaterOrEqual(key), key.arena());
+ }
+
+ MappedRangeResult v = wait(ryw->tr.getMappedRange(
+ read.begin, read.end, read.mapper, read.limits, snapshot, backwards ? Reverse::True : Reverse::False));
+ return v;
+ }
+
+ template
+ static void addConflictRangeAndMustUnmodified(ReadYourWritesTransaction* ryw,
+ GetMappedRangeReq read,
+ WriteMap::iterator& it,
+ MappedRangeResult result) {
+ // Primary getRange.
+ addConflictRange(
+ ryw, GetRangeReq(read.begin, read.end, read.limits), it, result);
+
+ // Secondary getValue/getRanges.
+ for (const auto& mappedKeyValue : result) {
+ const auto& reqAndResult = mappedKeyValue.reqAndResult;
+ if (std::holds_alternative(reqAndResult)) {
+ auto getValue = std::get(reqAndResult);
+ // GetValueReq variation of addConflictRange require it to point at the right segment.
+ it.skip(getValue.key);
+ // The result is not used in GetValueReq variation of addConflictRange. Let's just pass in a
+ // placeholder.
+ addConflictRange(ryw, GetValueReq(getValue.key), it, Optional());
+ } else if (std::holds_alternative(reqAndResult)) {
+ auto getRange = std::get(reqAndResult);
+ // We only support forward scan for secondary getRange requests.
+ // The limits are not used in addConflictRange. Let's just pass in a placeholder.
+ addConflictRange(
+ ryw, GetRangeReq(getRange.begin, getRange.end, GetRangeLimits()), it, getRange.result);
+ } else {
+ throw internal_error();
+ }
+ }
+ }
+
+ // For Snapshot::True and NOT readYourWritesDisabled.
+ ACTOR template
+ static Future readWithConflictRangeRYW(ReadYourWritesTransaction* ryw,
+ GetMappedRangeReq req,
+ Snapshot snapshot) {
+ choose {
+ when(MappedRangeResult result = wait(readThrough(ryw, req, Snapshot::True))) {
+ // Insert read conflicts (so that it supported Snapshot::True) and check it is not modified (so it masks
+ // sure not break RYW semantic while not implementing RYW) for both the primary getRange and all
+ // underlying getValue/getRanges.
+ WriteMap::iterator writes(&ryw->writes);
+ addConflictRangeAndMustUnmodified(ryw, req, writes, result);
+ return result;
+ }
+ when(wait(ryw->resetPromise.getFuture())) { throw internal_error(); }
+ }
+ }
+
+ template
+ static inline Future readWithConflictRangeForGetMappedRange(
+ ReadYourWritesTransaction* ryw,
+ GetMappedRangeReq const& req,
+ Snapshot snapshot) {
+ // For now, getMappedRange requires serializable isolation. (Technically it is trivial to add snapshot
+ // isolation support. But it is not default and is rarely used. So we disallow it until we have thorough test
+ // coverage for it.)
+ if (snapshot) {
+ TEST(true); // getMappedRange not supported for snapshot.
+ throw unsupported_operation();
+ }
+ // For now, getMappedRange requires read-your-writes being NOT disabled. But the support of RYW is limited
+ // to throwing get_mapped_range_reads_your_writes error when getMappedRange actually reads your own writes.
+ // Applications should fall back in their own ways. This is different from what is usually expected from RYW,
+ // which returns the written value transparently. In another word, it makes sure not break RYW semantics without
+ // actually implementing reading from the writes.
+ if (ryw->options.readYourWritesDisabled) {
+ TEST(true); // getMappedRange not supported for read-your-writes disabled.
+ throw unsupported_operation();
+ }
+
+ return readWithConflictRangeRYW(ryw, req, snapshot);
+ }
+
+#ifndef __INTEL_COMPILER
+#pragma endregion
+#endif
+
static void triggerWatches(ReadYourWritesTransaction* ryw,
KeyRangeRef range,
Optional val,
@@ -1571,16 +1668,16 @@ Future ReadYourWritesTransaction::getRange(const KeySelector& begin
return getRange(begin, end, GetRangeLimits(limit), snapshot, reverse);
}
-Future ReadYourWritesTransaction::getRangeAndFlatMap(KeySelector begin,
- KeySelector end,
- Key mapper,
- GetRangeLimits limits,
- Snapshot snapshot,
- Reverse reverse) {
+Future ReadYourWritesTransaction::getMappedRange(KeySelector begin,
+ KeySelector end,
+ Key mapper,
+ GetRangeLimits limits,
+ Snapshot snapshot,
+ Reverse reverse) {
if (getDatabase()->apiVersionAtLeast(630)) {
if (specialKeys.contains(begin.getKey()) && specialKeys.begin <= end.getKey() &&
end.getKey() <= specialKeys.end) {
- TEST(true); // Special key space get range (FlatMap)
+ TEST(true); // Special key space get range (getMappedRange)
throw client_invalid_operation(); // Not support special keys.
}
} else {
@@ -1602,8 +1699,8 @@ Future ReadYourWritesTransaction::getRangeAndFlatMap(KeySelector be
// This optimization prevents nullptr operations from being added to the conflict range
if (limits.isReached()) {
- TEST(true); // RYW range read limit 0 (FlatMap)
- return RangeResult();
+ TEST(true); // RYW range read limit 0 (getMappedRange)
+ return MappedRangeResult();
}
if (!limits.isValid())
@@ -1616,17 +1713,16 @@ Future ReadYourWritesTransaction::getRangeAndFlatMap(KeySelector be
end.removeOrEqual(end.arena());
if (begin.offset >= end.offset && begin.getKey() >= end.getKey()) {
- TEST(true); // RYW range inverted (FlatMap)
- return RangeResult();
+ TEST(true); // RYW range inverted (getMappedRange)
+ return MappedRangeResult();
}
- Future result =
- reverse ? RYWImpl::readWithConflictRangeAndFlatMap(
- this, RYWImpl::GetRangeAndFlatMapReq(begin, end, mapper, limits), snapshot)
- : RYWImpl::readWithConflictRangeAndFlatMap(
- this, RYWImpl::GetRangeAndFlatMapReq(begin, end, mapper, limits), snapshot);
+ Future result =
+ reverse ? RYWImpl::readWithConflictRangeForGetMappedRange(
+ this, RYWImpl::GetMappedRangeReq(begin, end, mapper, limits), snapshot)
+ : RYWImpl::readWithConflictRangeForGetMappedRange(
+ this, RYWImpl::GetMappedRangeReq(begin, end, mapper, limits), snapshot);
- reading.add(success(result));
return result;
}
@@ -1761,27 +1857,11 @@ void ReadYourWritesTransaction::addReadConflictRange(KeyRangeRef const& keys) {
}
void ReadYourWritesTransaction::updateConflictMap(KeyRef const& key, WriteMap::iterator& it) {
- // it.skip( key );
- // ASSERT( it.beginKey() <= key && key < it.endKey() );
- if (it.is_unmodified_range() || (it.is_operation() && !it.is_independent())) {
- approximateSize += 2 * key.expectedSize() + 1 + sizeof(KeyRangeRef);
- readConflicts.insert(singleKeyRange(key, arena), true);
- }
+ RYWImpl::updateConflictMap(this, key, it);
}
void ReadYourWritesTransaction::updateConflictMap(KeyRangeRef const& keys, WriteMap::iterator& it) {
- // it.skip( keys.begin );
- // ASSERT( it.beginKey() <= keys.begin && keys.begin < it.endKey() );
- for (; it.beginKey() < keys.end; ++it) {
- if (it.is_unmodified_range() || (it.is_operation() && !it.is_independent())) {
- KeyRangeRef insert_range = KeyRangeRef(std::max(keys.begin, it.beginKey().toArenaOrRef(arena)),
- std::min(keys.end, it.endKey().toArenaOrRef(arena)));
- if (!insert_range.empty()) {
- approximateSize += keys.expectedSize() + sizeof(KeyRangeRef);
- readConflicts.insert(insert_range, true);
- }
- }
- }
+ RYWImpl::updateConflictMap(this, keys, it);
}
void ReadYourWritesTransaction::writeRangeToNativeTransaction(KeyRangeRef const& keys) {
diff --git a/fdbclient/ReadYourWrites.h b/fdbclient/ReadYourWrites.h
index b8ccd23e549..84bc05e4ef7 100644
--- a/fdbclient/ReadYourWrites.h
+++ b/fdbclient/ReadYourWrites.h
@@ -61,6 +61,9 @@ struct TransactionDebugInfo : public ReferenceCounted {
// Values returned by a ReadYourWritesTransaction will contain a reference to the transaction's arena. Therefore,
// keeping a reference to a value longer than its creating transaction would hold all of the memory generated by the
// transaction
+// If options.readYourWritesDisabled, rely on NativeAPI to handle everything. Otherwise, read NativeAPI with
+// Snapshot::True and handle read conflicts at ReadYourWritesTransaction, write NativeAPI with AddConflictRange::False
+// and handle write conflicts at ReadYourWritesTransaction, eventually send this information to NativeAPI on commit.
class ReadYourWritesTransaction final : NonCopyable,
public ISingleThreadTransaction,
public FastAllocated {
@@ -104,12 +107,12 @@ class ReadYourWritesTransaction final : NonCopyable,
snapshot,
reverse);
}
- Future getRangeAndFlatMap(KeySelector begin,
- KeySelector end,
- Key mapper,
- GetRangeLimits limits,
- Snapshot = Snapshot::False,
- Reverse = Reverse::False) override;
+ Future getMappedRange(KeySelector begin,
+ KeySelector end,
+ Key mapper,
+ GetRangeLimits limits,
+ Snapshot = Snapshot::False,
+ Reverse = Reverse::False) override;
[[nodiscard]] Future>> getAddressesForKey(const Key& key) override;
Future>> getRangeSplitPoints(const KeyRange& range, int64_t chunkSize) override;
diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp
index 6ec7778f8e1..a65099edd52 100644
--- a/fdbclient/ServerKnobs.cpp
+++ b/fdbclient/ServerKnobs.cpp
@@ -676,8 +676,10 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( MAX_STORAGE_COMMIT_TIME, 120.0 ); //The max fsync stall time on the storage server and tlog before marking a disk as failed
init( RANGESTREAM_LIMIT_BYTES, 2e6 ); if( randomize && BUGGIFY ) RANGESTREAM_LIMIT_BYTES = 1;
init( ENABLE_CLEAR_RANGE_EAGER_READS, true );
- init( QUICK_GET_VALUE_FALLBACK, false );
- init( QUICK_GET_KEY_VALUES_FALLBACK, false );
+ init( QUICK_GET_VALUE_FALLBACK, true );
+ init( QUICK_GET_KEY_VALUES_FALLBACK, true );
+ init( QUICK_GET_KEY_VALUES_LIMIT, 2000 );
+ init( QUICK_GET_KEY_VALUES_LIMIT_BYTES, 1e7 );
//Wait Failure
init( MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS, 250 ); if( randomize && BUGGIFY ) MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS = 2;
diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h
index 3e25bbe7fee..f962ec78b9e 100644
--- a/fdbclient/ServerKnobs.h
+++ b/fdbclient/ServerKnobs.h
@@ -615,6 +615,8 @@ class ServerKnobs : public KnobsImpl {
bool ENABLE_CLEAR_RANGE_EAGER_READS;
bool QUICK_GET_VALUE_FALLBACK;
bool QUICK_GET_KEY_VALUES_FALLBACK;
+ int QUICK_GET_KEY_VALUES_LIMIT;
+ int QUICK_GET_KEY_VALUES_LIMIT_BYTES;
// Wait Failure
int MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS;
diff --git a/fdbclient/SimpleConfigTransaction.h b/fdbclient/SimpleConfigTransaction.h
index 168b1a6c294..871d0efa971 100644
--- a/fdbclient/SimpleConfigTransaction.h
+++ b/fdbclient/SimpleConfigTransaction.h
@@ -59,12 +59,12 @@ class SimpleConfigTransaction final : public IConfigTransaction, public FastAllo
GetRangeLimits limits,
Snapshot = Snapshot::False,
Reverse = Reverse::False) override;
- Future getRangeAndFlatMap(KeySelector begin,
- KeySelector end,
- Key mapper,
- GetRangeLimits limits,
- Snapshot = Snapshot::False,
- Reverse = Reverse::False) override {
+ Future getMappedRange(KeySelector begin,
+ KeySelector end,
+ Key mapper,
+ GetRangeLimits limits,
+ Snapshot = Snapshot::False,
+ Reverse = Reverse::False) override {
throw client_invalid_operation();
}
Future commit() override;
diff --git a/fdbclient/StorageServerInterface.cpp b/fdbclient/StorageServerInterface.cpp
index 3c389b5ab42..dce4df77361 100644
--- a/fdbclient/StorageServerInterface.cpp
+++ b/fdbclient/StorageServerInterface.cpp
@@ -156,20 +156,20 @@ void TSS_traceMismatch(TraceEvent& event,
// range reads and flat map
template <>
-bool TSS_doCompare(const GetKeyValuesAndFlatMapReply& src, const GetKeyValuesAndFlatMapReply& tss) {
+bool TSS_doCompare(const GetMappedKeyValuesReply& src, const GetMappedKeyValuesReply& tss) {
return src.more == tss.more && src.data == tss.data;
}
template <>
-const char* TSS_mismatchTraceName(const GetKeyValuesAndFlatMapRequest& req) {
- return "TSSMismatchGetKeyValuesAndFlatMap";
+const char* TSS_mismatchTraceName(const GetMappedKeyValuesRequest& req) {
+ return "TSSMismatchGetMappedKeyValues";
}
template <>
void TSS_traceMismatch(TraceEvent& event,
- const GetKeyValuesAndFlatMapRequest& req,
- const GetKeyValuesAndFlatMapReply& src,
- const GetKeyValuesAndFlatMapReply& tss) {
+ const GetMappedKeyValuesRequest& req,
+ const GetMappedKeyValuesReply& src,
+ const GetMappedKeyValuesReply& tss) {
std::string ssResultsString = format("(%d)%s:\n", src.data.size(), src.more ? "+" : "");
for (auto& it : src.data) {
ssResultsString += "\n" + it.key.printable() + "=" + traceChecksumValue(it.value);
@@ -400,9 +400,9 @@ void TSSMetrics::recordLatency(const GetKeyValuesRequest& req, double ssLatency,
}
template <>
-void TSSMetrics::recordLatency(const GetKeyValuesAndFlatMapRequest& req, double ssLatency, double tssLatency) {
- SSgetKeyValuesAndFlatMapLatency.addSample(ssLatency);
- TSSgetKeyValuesAndFlatMapLatency.addSample(tssLatency);
+void TSSMetrics::recordLatency(const GetMappedKeyValuesRequest& req, double ssLatency, double tssLatency) {
+ SSgetMappedKeyValuesLatency.addSample(ssLatency);
+ TSSgetMappedKeyValuesLatency.addSample(tssLatency);
}
template <>
diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h
index aef7486c3d6..75c9411f18e 100644
--- a/fdbclient/StorageServerInterface.h
+++ b/fdbclient/StorageServerInterface.h
@@ -67,7 +67,7 @@ struct StorageServerInterface {
// Throws a wrong_shard_server if the keys in the request or result depend on data outside this server OR if a large
// selector offset prevents all data from being read in one range read
RequestStream getKeyValues;
- RequestStream getKeyValuesAndFlatMap;
+ RequestStream getMappedKeyValues;
RequestStream getShardState;
RequestStream waitMetrics;
@@ -127,8 +127,8 @@ struct StorageServerInterface {
RequestStream(getValue.getEndpoint().getAdjustedEndpoint(12));
getKeyValuesStream =
RequestStream(getValue.getEndpoint().getAdjustedEndpoint(13));
- getKeyValuesAndFlatMap =
- RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14));
+ getMappedKeyValues =
+ RequestStream(getValue.getEndpoint().getAdjustedEndpoint(14));
changeFeedStream =
RequestStream(getValue.getEndpoint().getAdjustedEndpoint(15));
overlappingChangeFeeds =
@@ -179,7 +179,7 @@ struct StorageServerInterface {
streams.push_back(getReadHotRanges.getReceiver());
streams.push_back(getRangeSplitPoints.getReceiver());
streams.push_back(getKeyValuesStream.getReceiver(TaskPriority::LoadBalancedEndpoint));
- streams.push_back(getKeyValuesAndFlatMap.getReceiver(TaskPriority::LoadBalancedEndpoint));
+ streams.push_back(getMappedKeyValues.getReceiver(TaskPriority::LoadBalancedEndpoint));
streams.push_back(changeFeedStream.getReceiver());
streams.push_back(overlappingChangeFeeds.getReceiver());
streams.push_back(changeFeedPop.getReceiver());
@@ -362,15 +362,17 @@ struct GetKeyValuesRequest : TimedRequest {
}
};
-struct GetKeyValuesAndFlatMapReply : public LoadBalancedReply {
+struct GetMappedKeyValuesReply : public LoadBalancedReply {
constexpr static FileIdentifier file_identifier = 1783067;
Arena arena;
- VectorRef data;
+ // MappedKeyValueRef is not string_serialized_traits, so we have to use FlatBuffers.
+ VectorRef data;
+
Version version; // useful when latestVersion was requested
bool more;
bool cached = false;
- GetKeyValuesAndFlatMapReply() : version(invalidVersion), more(false), cached(false) {}
+ GetMappedKeyValuesReply() : version(invalidVersion), more(false), cached(false) {}
template
void serialize(Ar& ar) {
@@ -378,7 +380,7 @@ struct GetKeyValuesAndFlatMapReply : public LoadBalancedReply {
}
};
-struct GetKeyValuesAndFlatMapRequest : TimedRequest {
+struct GetMappedKeyValuesRequest : TimedRequest {
constexpr static FileIdentifier file_identifier = 6795747;
SpanID spanContext;
Arena arena;
@@ -390,10 +392,9 @@ struct GetKeyValuesAndFlatMapRequest : TimedRequest {
bool isFetchKeys;
Optional tags;
Optional debugID;
- ReplyPromise reply;
-
- GetKeyValuesAndFlatMapRequest() : isFetchKeys(false) {}
+ ReplyPromise reply;
+ GetMappedKeyValuesRequest() : isFetchKeys(false) {}
template
void serialize(Ar& ar) {
serializer(ar,
diff --git a/fdbclient/ThreadSafeTransaction.cpp b/fdbclient/ThreadSafeTransaction.cpp
index eb50b0f16f5..ce2d6c39c4c 100644
--- a/fdbclient/ThreadSafeTransaction.cpp
+++ b/fdbclient/ThreadSafeTransaction.cpp
@@ -258,20 +258,20 @@ ThreadFuture ThreadSafeTransaction::getRange(const KeySelectorRef&
});
}
-ThreadFuture