ZooKeeper
  1. ZooKeeper
  2. ZOOKEEPER-1568

multi should have a non-transaction version

    Details

    • Type: Improvement Improvement
    • Status: Open
    • Priority: Major Major
    • Resolution: Unresolved
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      Currently multi is transactional, i.e. all or none. However, sometimes, we don't want that. We want all operations to be executed. Even some operation(s) fails, it is ok. We just need to know the result of each operation.

      1. zk-1568_v2.patch
        34 kB
        Jimmy Xiang
      2. zk-1568_v1.patch
        16 kB
        Jimmy Xiang

        Issue Links

          Activity

          Hide
          Flavio Junqueira added a comment -

          Hi Jimmy, I'm trying to understand why submitting operations asynchronously is not sufficient for your case. Why do you need to use multi in this case?

          Show
          Flavio Junqueira added a comment - Hi Jimmy, I'm trying to understand why submitting operations asynchronously is not sufficient for your case. Why do you need to use multi in this case?
          Hide
          Jimmy Xiang added a comment -

          Hi Flavio, for our use case, we need to create/setData hundreds/thousands of znodes. By submitting operations asynchronously, we need to do it one by one. If we can do it in batches, we can save lots of network trips.

          Show
          Jimmy Xiang added a comment - Hi Flavio, for our use case, we need to create/setData hundreds/thousands of znodes. By submitting operations asynchronously, we need to do it one by one. If we can do it in batches, we can save lots of network trips.
          Hide
          Flavio Junqueira added a comment -

          In my view, the asynchronous API has been designed to address exactly use cases like yours. I don't think you should be suffering any severe penalty by using the asynchronous API. Have you actually tried it and had any issue with it?

          Show
          Flavio Junqueira added a comment - In my view, the asynchronous API has been designed to address exactly use cases like yours. I don't think you should be suffering any severe penalty by using the asynchronous API. Have you actually tried it and had any issue with it?
          Hide
          Jimmy Xiang added a comment -

          Yes, that's what we are using now. It is working fine. I was thinking if there is still room for improvement.

          Show
          Jimmy Xiang added a comment - Yes, that's what we are using now. It is working fine. I was thinking if there is still room for improvement.
          Hide
          Marshall McMullen added a comment -

          I actually think there is a valid use case for this. Mostly for performance reasons. Because a multi is one transaction, it causes less permuation on the distributed and replicated state of zookeeper than multiple individual operations not in a multi.

          With a Multi:

          • You only pay the cost of the RPC overhead once rather than on each individual operation
          • You get one flush of the leader channel rather than multiple ones for each write operation
          • A multi will case one new snapshot/log to be generated rather than multiple ones for each operation

          There are other reasons that make this a good reason too that are not performance based. e.g., if it makes the programmer's job easier to use a multi with these semantics, then that's a win.

          In other distributed databases I've worked on, we used different terminology to disinguish between a multi op that all succeed/fail vs one that does not. We used the term "Batch" to imply we were batching up operations but there was no guarantee they'd all succeed/fail.

          Show
          Marshall McMullen added a comment - I actually think there is a valid use case for this. Mostly for performance reasons. Because a multi is one transaction, it causes less permuation on the distributed and replicated state of zookeeper than multiple individual operations not in a multi. With a Multi: You only pay the cost of the RPC overhead once rather than on each individual operation You get one flush of the leader channel rather than multiple ones for each write operation A multi will case one new snapshot/log to be generated rather than multiple ones for each operation There are other reasons that make this a good reason too that are not performance based. e.g., if it makes the programmer's job easier to use a multi with these semantics, then that's a win. In other distributed databases I've worked on, we used different terminology to disinguish between a multi op that all succeed/fail vs one that does not. We used the term "Batch" to imply we were batching up operations but there was no guarantee they'd all succeed/fail.
          Hide
          Ted Yu added a comment -

          A multi will case one new snapshot/log to be generated

          I guess you meant 'cause' above.

          but there was no guarantee they'd all succeed/fail.

          I think we need to formalize how success / failure status for individual operations in this new multi API should be delivered back to client.

          Show
          Ted Yu added a comment - A multi will case one new snapshot/log to be generated I guess you meant 'cause' above. but there was no guarantee they'd all succeed/fail. I think we need to formalize how success / failure status for individual operations in this new multi API should be delivered back to client.
          Hide
          Marshall McMullen added a comment -

          Yes, I meant 'cause' .

          The existing multi code fills in a list of results for each op. Right now, it aborts on the first op that fails and rolls back the data tree to what it was before it started. And it explicitly marks all ops after that in the results list with a runtime exception. So the mechanism is already there to communicate the errors back to the client.

          So I suppose the Multi code would need to take a bool to indicate if it was all or nothing or not.

          Show
          Marshall McMullen added a comment - Yes, I meant 'cause' . The existing multi code fills in a list of results for each op. Right now, it aborts on the first op that fails and rolls back the data tree to what it was before it started. And it explicitly marks all ops after that in the results list with a runtime exception. So the mechanism is already there to communicate the errors back to the client. So I suppose the Multi code would need to take a bool to indicate if it was all or nothing or not.
          Hide
          Ted Yu added a comment -

          it aborts on the first op that fails and rolls back

          Should we allow operations after the failed operation to continue ?
          The rationale is that the operations in the batch may not have dependencies among them.

          Show
          Ted Yu added a comment - it aborts on the first op that fails and rolls back Should we allow operations after the failed operation to continue ? The rationale is that the operations in the batch may not have dependencies among them.
          Hide
          Marshall McMullen added a comment -

          Well obviously the way it is currently implemented we do not proceed past the first failure. But if we wanted to support a "batch" request wherein they are not all or nothing, then yes, I think we'd proceed past the first failure. If there are dependencies on earlier ops, then obviously those will fail.

          Show
          Marshall McMullen added a comment - Well obviously the way it is currently implemented we do not proceed past the first failure. But if we wanted to support a "batch" request wherein they are not all or nothing, then yes, I think we'd proceed past the first failure. If there are dependencies on earlier ops, then obviously those will fail.
          Hide
          Jimmy Xiang added a comment -

          For our use case, there is no dependency issue. "Batch" is what we want. Another benefit is that we will get less ZK events. This may not be obvious. But we will do get less ZK events. For example, for a bunch of create operations, if we do it one by one, once we get a nodeChildrenChanged event, we will watch it again, so we will get another one for the next create operation. If they are batched, after we get the first nodeChildrenChanged event, when we watch it again, most likely, other nodes are already created, so we will get less events, which is good.

          Show
          Jimmy Xiang added a comment - For our use case, there is no dependency issue. "Batch" is what we want. Another benefit is that we will get less ZK events. This may not be obvious. But we will do get less ZK events. For example, for a bunch of create operations, if we do it one by one, once we get a nodeChildrenChanged event, we will watch it again, so we will get another one for the next create operation. If they are batched, after we get the first nodeChildrenChanged event, when we watch it again, most likely, other nodes are already created, so we will get less events, which is good.
          Hide
          Flavio Junqueira added a comment -

          Hi guys, Here are two concerns I have:

          1. With this proposal we are now mixing performance and correctness (atomicity) in the multi abstraction. At this point, I'd rather stick only to the correctness aspect.
          2. The architecture of zookeeper is essentially an execution pipeline, which has been optimized to provide both low latency and high throughput. This proposal goes in the opposite the direction and tries to promote the execution of large batches instead of individual operations at least for some use cases.

          In general, if it there is an opportunity to improve the performance of the system, then we should pursue it, but at this point it is not even clear how much difference it would actually make if any. Can we actually make sure that such an app-level batching makes a significant difference compared to trunk with respect to performance? And if it does, what exactly is the culprit? Can we fix it without introducing a new API feature?

          The point about getChildren capturing fewer events sounds like a "good to have" but not really "must have", but please correct me if I'm wrong.

          Show
          Flavio Junqueira added a comment - Hi guys, Here are two concerns I have: With this proposal we are now mixing performance and correctness (atomicity) in the multi abstraction. At this point, I'd rather stick only to the correctness aspect. The architecture of zookeeper is essentially an execution pipeline, which has been optimized to provide both low latency and high throughput. This proposal goes in the opposite the direction and tries to promote the execution of large batches instead of individual operations at least for some use cases. In general, if it there is an opportunity to improve the performance of the system, then we should pursue it, but at this point it is not even clear how much difference it would actually make if any. Can we actually make sure that such an app-level batching makes a significant difference compared to trunk with respect to performance? And if it does, what exactly is the culprit? Can we fix it without introducing a new API feature? The point about getChildren capturing fewer events sounds like a "good to have" but not really "must have", but please correct me if I'm wrong.
          Hide
          Jimmy Xiang added a comment -

          Attached is the first version patch which is on top of the patch for ZK-1569 and ZK-1592.

          This patch supports sync-batch only. The async version will be in the next patch.

          Show
          Jimmy Xiang added a comment - Attached is the first version patch which is on top of the patch for ZK-1569 and ZK-1592. This patch supports sync-batch only. The async version will be in the next patch.
          Hide
          Flavio Junqueira added a comment -

          I had a look at the patch, and I'm not sure how you take care of the scenario you mention above of receiving fewer zk events. With this patch, wouldn't you still get as many notifications as watches you had set for the znodes you have manipulated in your batch?

          I'm still interested in understanding if the performance difference you claim matters or of it can be fixed some other way. This feature is mainly for performance and convenience, yes?

          Show
          Flavio Junqueira added a comment - I had a look at the patch, and I'm not sure how you take care of the scenario you mention above of receiving fewer zk events. With this patch, wouldn't you still get as many notifications as watches you had set for the znodes you have manipulated in your batch? I'm still interested in understanding if the performance difference you claim matters or of it can be fixed some other way. This feature is mainly for performance and convenience, yes?
          Hide
          Jimmy Xiang added a comment -

          I am hacking around this week. I will find out how much performance I can gain this way, if any. Yes, it is mainly for performance and convenience.
          As to fewer zk events, it may be just for our use case, assuming our node children changed handler doesn't have a chance to reset the watch soon enough. So if we create lots of children for one parent, we may get less node children changed events, theoretically.

          Show
          Jimmy Xiang added a comment - I am hacking around this week. I will find out how much performance I can gain this way, if any. Yes, it is mainly for performance and convenience. As to fewer zk events, it may be just for our use case, assuming our node children changed handler doesn't have a chance to reset the watch soon enough. So if we create lots of children for one parent, we may get less node children changed events, theoretically.
          Hide
          Jimmy Xiang added a comment -

          Patch version 2 which supports async batch.

          Show
          Jimmy Xiang added a comment - Patch version 2 which supports async batch.
          Hide
          Patrick Hunt added a comment -

          Jimmy this ready for review/commit? If so please "submit". Thanks.

          Show
          Patrick Hunt added a comment - Jimmy this ready for review/commit? If so please "submit". Thanks.

            People

            • Assignee:
              Unassigned
              Reporter:
              Jimmy Xiang
            • Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

              • Created:
                Updated:

                Development