BewareMyPower commented on PR #21271: URL: https://github.com/apache/pulsar/pull/21271#issuecomment-1751899833
I see @merlimat's suggestion [here](https://github.com/apache/pulsar/pull/21166#issuecomment-1719694532) and you respond: https://github.com/apache/pulsar/pull/21166#issuecomment-1720430117 > Even if we refresh the tableView before each data read, we can't guarantee that the data retrieved is the most up-to-date. With this proposal, we still cannot guarantee the "latest" value is retrieved. Just like you mentioned, this proposal only guarantees the value is latest at the checkpoint when `refreshAsync` is called. > This configuration might confuse users. I disagree. Assume we added two options: `STRONG_CONSISTENCY_MODEL` and `EVENTUAL_CONSISTENCY_MODEL`), then we'll have: - `STRONG_CONSISTENCY_MODEL`: any method will be blocked until the latest value is retrieved. - `EVENTUAL_CONSISTENCY_MODEL`: all methods are non-blocking but the value retrieved might not be latest at the time point. This pattern is similar to the [POSIX `read` API](https://man7.org/linux/man-pages/man2/read.2.html), which is widely used by system development. It's blocking by default, i.e. it will be blocked until some data is available. But if the file descriptor is marked as `O_NONBLOCK`, it will become non-blocking. > We would need to add many asynchronous interfaces, such as T getAsync(String key); boolean containsKeyAsync(String key); Collection valuesAsync(); etc. I think the main concern is that `get` might be blocking while Pulsar uses the `future.xxxAsync` everywhere and it's dangerous to call a blocking method in the callback of `xxxAsync`. For the sake of this reason, I'm +1 to add a method to read messages to latest at a certain time point. However, I think the `refreshAsync` design is not good. Considering the following case: ```java view.refreshAsync().thenAccept(__ -> { // Now, the TableView is the latest snapshot of the compacted topic if (view.containsKey(key)) { // T1 process(view.get(key)); // T2 } }); ``` Assume at `T1`, the internal data is `{ key => value1 }`. Since the `TableView` keeps fetching the latest message at the background, if a new message `(key, value2)` is written after `T1` and before `T2`, at `T2`, `view.get(key)` will be `value2`. The root cause is that `TableView` is not immutable. `TableViewImpl#readTailMessages` fetches the messages at the background. So I suggest returning an immutable map like: ```java /** * Get the latest snapshot for a set of keys at the current time point. * * @param keys * @return a future of the map that represents the snapshot of the table view. The keys must be a subset of the * `keys` parameter and the value is guaranteed to be the latest value before the future is completed. */ CompletableFuture<Map<String, T>> getLatestSnapshotAsync(Set<String> keys); /** * Get the latest snapshot at the current time point. * @return a future of map that represents the snapshot of the table view. */ CompletableFuture<Map<String, T>> getLatestSnapshotAsync(); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
