You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by an...@apache.org on 2019/06/26 08:57:53 UTC

[zookeeper] branch master updated: ZOOKEEPER-3402: Add multiRead operation

This is an automated email from the ASF dual-hosted git repository.

andor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 1b074d6  ZOOKEEPER-3402: Add multiRead operation
1b074d6 is described below

commit 1b074d6a8a448e1d3d50383a7d6f2e21d5cf1dfc
Author: szepet <sz...@gmail.com>
AuthorDate: Wed Jun 26 10:57:46 2019 +0200

    ZOOKEEPER-3402: Add multiRead operation
    
    So, the possibility of the multi version of getChildren, getData, and a common multiRead interface was already mentioned and briefly discussed in the conversation thread of #922.
    This patch introduces the concept of multiRead operation.
    Note: This is a 'work-in-progress', not every comment is added, and test cases should be extended as well. Currently, the aim is to decide whether this direction is something that the community would get behind or should fin.
    I've faced more design decision while implementing this (note: I wanted to keep the backward compatibility at all cost), the more important ones were the following:
    1. Whether the new read operations should be part of Op or should we create new descendant classes of `Op` like `ReadOp` and `WriteOp`. In this case, I believe it would be unnecessary to create new classes since it would create a lot of code duplication (despite the common ancestor) and the flow of the processing would not be as clean as in the initial state. However, by simply adding an OpKind enum to the class enables us to ensure the type-safety behavior (read and write operations  [...]
    2. At which point of request processing should we capture the mixed (transaction and read operation) multi requests? It seems pretty clear to catch them client side and not create extra work for the server.
    3. Do we even need a `multiRead` operation or this whole improvement could be just part of multi? Well, the main problem is the following: currently, on server side (and in general, everywhere in ZooKeeper) the fact that multi only contains transactions is a heavily used and built upon it. The whole `multi` processing flow should be rewritten from zero and also the server would get a significant extra work for that. So yeah, it seems pretty clear to me that introducing the `multiRead` [...]
    
    Hope these thoughts make sense! Any observations, questions are welcome!
    
    Author: szepet <sz...@gmail.com>
    Author: Peter Szecsi <sz...@gmail.com>
    
    Reviewers: fangmin@apache.org, andor@apache.org
    
    Closes #959 from szepet/ZOOKEEPER-3402 and squashes the following commits:
    
    2f128ef11 [szepet] rephrase exception
    660351cce [Peter Szecsi] Rename MultiTransactionRecord to MultiOperationRecord
    71a6d2e22 [Peter Szecsi] address some review comments
    3cec54850 [szepet] make sure not to expose the internal representation in GetDataResult
    d4c6764ee [szepet] ZOOKEEPER-3402: Add multiRead operation
---
 .../org/apache/zookeeper/MultiOperationRecord.java | 199 ++++++++++++++++++
 .../java/org/apache/zookeeper/MultiResponse.java   |  20 ++
 .../apache/zookeeper/MultiTransactionRecord.java   | 163 ---------------
 .../src/main/java/org/apache/zookeeper/Op.java     | 102 +++++++++-
 .../main/java/org/apache/zookeeper/OpResult.java   |  72 +++++++
 .../main/java/org/apache/zookeeper/ZooDefs.java    |   2 +
 .../main/java/org/apache/zookeeper/ZooKeeper.java  |  40 +++-
 .../zookeeper/server/FinalRequestProcessor.java    |  88 ++++++--
 .../zookeeper/server/PrepRequestProcessor.java     |   7 +-
 .../java/org/apache/zookeeper/server/Request.java  |   4 +
 .../apache/zookeeper/server/TraceFormatter.java    |   2 +
 .../server/quorum/QuorumZooKeeperServer.java       |   4 +-
 ...cordTest.java => MultiOperationRecordTest.java} |  14 +-
 .../zookeeper/server/PrepRequestProcessorTest.java |   4 +-
 ...ransactionTest.java => MultiOperationTest.java} | 222 ++++++++++++++++++++-
 15 files changed, 721 insertions(+), 222 deletions(-)

diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/MultiOperationRecord.java b/zookeeper-server/src/main/java/org/apache/zookeeper/MultiOperationRecord.java
new file mode 100644
index 0000000..ef6a9f3
--- /dev/null
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/MultiOperationRecord.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.zookeeper;
+
+import org.apache.jute.InputArchive;
+import org.apache.jute.OutputArchive;
+import org.apache.jute.Record;
+import org.apache.zookeeper.proto.*;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Encodes a composite operation.  In the wire format, each operation
+ * consists of a single MultiHeader followed by the appropriate request.
+ * Each of these MultiHeaders has a type which indicates
+ * the type of the following operation or a negative number if no more operations
+ * are included.
+ * All of the operations must be from the same OpKind.
+ */
+public class MultiOperationRecord implements Record, Iterable<Op> {
+    private List<Op> ops = new ArrayList<Op>();
+    private Op.OpKind opKind = null;
+
+    public MultiOperationRecord() {
+    }
+
+    public MultiOperationRecord(Iterable<Op> ops) throws IllegalArgumentException {
+        for (Op op : ops) {
+            setOrCheckOpKind(op.getKind());
+            add(op);
+        }
+    }
+
+    @Override
+    public Iterator<Op> iterator() {
+        return ops.iterator() ;
+    }
+
+    public void add(Op op) throws IllegalArgumentException {
+        setOrCheckOpKind(op.getKind());
+        ops.add(op);
+    }
+
+    public int size() {
+        return ops.size();
+    }
+
+    /**
+     * Returns the kind of the operations contained by the record.
+     * @return  The OpKind value of all the elements in the record.
+     */
+    public Op.OpKind getOpKind() {
+        return opKind;
+    }
+
+    private void setOrCheckOpKind(Op.OpKind ok) throws IllegalArgumentException {
+        if (opKind == null) {
+            opKind = ok;
+        } else if (ok != opKind) {
+            throw new IllegalArgumentException("Mixing read and write operations (transactions)" +
+                      " is not allowed in a multi request.");
+        }
+    }
+
+    @Override
+    public void serialize(OutputArchive archive, String tag) throws IOException {
+        archive.startRecord(this, tag);
+        for (Op op : ops) {
+            MultiHeader h = new MultiHeader(op.getType(), false, -1);
+            h.serialize(archive, tag);
+            switch (op.getType()) {
+                case ZooDefs.OpCode.create:
+                case ZooDefs.OpCode.create2:
+                case ZooDefs.OpCode.createTTL:
+                case ZooDefs.OpCode.createContainer:
+                case ZooDefs.OpCode.delete:
+                case ZooDefs.OpCode.setData:
+                case ZooDefs.OpCode.check:
+                case ZooDefs.OpCode.getChildren:
+                case ZooDefs.OpCode.getData:
+                    op.toRequestRecord().serialize(archive, tag);
+                    break;
+                default:
+                    throw new IOException("Invalid type of op");
+            }
+        }
+        new MultiHeader(-1, true, -1).serialize(archive, tag);
+        archive.endRecord(this, tag);
+    }
+
+    @Override
+    public void deserialize(InputArchive archive, String tag) throws IOException {
+        archive.startRecord(tag);
+        MultiHeader h = new MultiHeader();
+        h.deserialize(archive, tag);
+        try {
+            while (!h.getDone()) {
+                switch (h.getType()) {
+                    case ZooDefs.OpCode.create:
+                    case ZooDefs.OpCode.create2:
+                    case ZooDefs.OpCode.createContainer:
+                        CreateRequest cr = new CreateRequest();
+                        cr.deserialize(archive, tag);
+                        add(Op.create(cr.getPath(), cr.getData(), cr.getAcl(), cr.getFlags()));
+                        break;
+                    case ZooDefs.OpCode.createTTL:
+                        CreateTTLRequest crTtl = new CreateTTLRequest();
+                        crTtl.deserialize(archive, tag);
+                        add(Op.create(crTtl.getPath(), crTtl.getData(), crTtl.getAcl(), crTtl.getFlags(), crTtl.getTtl()));
+                        break;
+                    case ZooDefs.OpCode.delete:
+                        DeleteRequest dr = new DeleteRequest();
+                        dr.deserialize(archive, tag);
+                        add(Op.delete(dr.getPath(), dr.getVersion()));
+                        break;
+                    case ZooDefs.OpCode.setData:
+                        SetDataRequest sdr = new SetDataRequest();
+                        sdr.deserialize(archive, tag);
+                        add(Op.setData(sdr.getPath(), sdr.getData(), sdr.getVersion()));
+                        break;
+                    case ZooDefs.OpCode.check:
+                        CheckVersionRequest cvr = new CheckVersionRequest();
+                        cvr.deserialize(archive, tag);
+                        add(Op.check(cvr.getPath(), cvr.getVersion()));
+                        break;
+                    case ZooDefs.OpCode.getChildren:
+                        GetChildrenRequest gcr = new GetChildrenRequest();
+                        gcr.deserialize(archive, tag);
+                        add(Op.getChildren(gcr.getPath()));
+                        break;
+                    case ZooDefs.OpCode.getData:
+                        GetDataRequest gdr = new GetDataRequest();
+                        gdr.deserialize(archive, tag);
+                        add(Op.getData(gdr.getPath()));
+                        break;
+                    default:
+                        throw new IOException("Invalid type of op");
+                }
+                h.deserialize(archive, tag);
+            }
+        } catch (IllegalArgumentException e) {
+            throw new IOException("Mixing different kind of ops");
+        }
+        archive.endRecord(tag);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof MultiOperationRecord)) return false;
+
+        MultiOperationRecord that = (MultiOperationRecord) o;
+
+        if (ops != null) {
+            Iterator<Op> other = that.ops.iterator();
+            for (Op op : ops) {
+                boolean hasMoreData = other.hasNext();
+                if (!hasMoreData) {
+                    return false;
+                }
+                Op otherOp = other.next();
+                if (!op.equals(otherOp)) {
+                    return false;
+                }
+            }
+            return !other.hasNext();
+        } else {
+            return that.ops == null;
+        }
+
+    }
+
+    @Override
+    public int hashCode() {
+        int h = 1023;
+        for (Op op : ops) {
+            h = h * 25 + op.hashCode();
+        }
+        return h;
+    }
+}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/MultiResponse.java b/zookeeper-server/src/main/java/org/apache/zookeeper/MultiResponse.java
index 5ac906a..bd5c77b 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/MultiResponse.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/MultiResponse.java
@@ -22,6 +22,8 @@ import org.apache.jute.OutputArchive;
 import org.apache.jute.Record;
 import org.apache.zookeeper.proto.Create2Response;
 import org.apache.zookeeper.proto.CreateResponse;
+import org.apache.zookeeper.proto.GetChildrenResponse;
+import org.apache.zookeeper.proto.GetDataResponse;
 import org.apache.zookeeper.proto.MultiHeader;
 import org.apache.zookeeper.proto.SetDataResponse;
 import org.apache.zookeeper.proto.ErrorResponse;
@@ -78,6 +80,12 @@ public class MultiResponse implements Record, Iterable<OpResult> {
                 case ZooDefs.OpCode.setData:
                     new SetDataResponse(((OpResult.SetDataResult) result).getStat()).serialize(archive, tag);
                     break;
+                case ZooDefs.OpCode.getChildren:
+                    new GetChildrenResponse(((OpResult.GetChildrenResult) result).getChildren()).serialize(archive, tag);
+                    break;
+                case ZooDefs.OpCode.getData:
+                    new GetDataResponse(((OpResult.GetDataResult) result).getData(),((OpResult.GetDataResult) result).getStat()).serialize(archive, tag);
+                    break;
                 case ZooDefs.OpCode.error:
                     new ErrorResponse(((OpResult.ErrorResult) result).getErr()).serialize(archive, tag);
                     break;
@@ -124,6 +132,18 @@ public class MultiResponse implements Record, Iterable<OpResult> {
                     results.add(new OpResult.CheckResult());
                     break;
 
+                case ZooDefs.OpCode.getChildren:
+                    GetChildrenResponse gcr = new GetChildrenResponse();
+                    gcr.deserialize(archive, tag);
+                    results.add(new OpResult.GetChildrenResult(gcr.getChildren()));
+                    break;
+
+                case ZooDefs.OpCode.getData:
+                    GetDataResponse gdr = new GetDataResponse();
+                    gdr.deserialize(archive, tag);
+                    results.add(new OpResult.GetDataResult(gdr.getData(), gdr.getStat()));
+                    break;
+
                 case ZooDefs.OpCode.error:
                     //FIXME: need way to more cleanly serialize/deserialize exceptions
                     ErrorResponse er = new ErrorResponse();
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/MultiTransactionRecord.java b/zookeeper-server/src/main/java/org/apache/zookeeper/MultiTransactionRecord.java
deleted file mode 100644
index 336a677..0000000
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/MultiTransactionRecord.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You 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 org.apache.zookeeper;
-
-import org.apache.jute.InputArchive;
-import org.apache.jute.OutputArchive;
-import org.apache.jute.Record;
-import org.apache.zookeeper.proto.*;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * Encodes a composite transaction.  In the wire format, each transaction
- * consists of a single MultiHeader followed by the appropriate request.
- * Each of these MultiHeaders has a type which indicates
- * the type of the following transaction or a negative number if no more transactions
- * are included.
- */
-public class MultiTransactionRecord implements Record, Iterable<Op> {
-    private List<Op> ops = new ArrayList<Op>();
-
-    public MultiTransactionRecord() {
-    }
-
-    public MultiTransactionRecord(Iterable<Op> ops) {
-        for (Op op : ops) {
-            add(op);
-        }
-    }
-
-    @Override
-    public Iterator<Op> iterator() {
-        return ops.iterator() ;
-    }
-
-    public void add(Op op) {
-        ops.add(op);
-    }
-
-    public int size() {
-        return ops.size();
-    }
-
-    @Override
-    public void serialize(OutputArchive archive, String tag) throws IOException {
-        archive.startRecord(this, tag);
-        for (Op op : ops) {
-            MultiHeader h = new MultiHeader(op.getType(), false, -1);
-            h.serialize(archive, tag);
-            switch (op.getType()) {
-                case ZooDefs.OpCode.create:
-                case ZooDefs.OpCode.create2:
-                case ZooDefs.OpCode.createTTL:
-                case ZooDefs.OpCode.createContainer:
-                case ZooDefs.OpCode.delete:
-                case ZooDefs.OpCode.setData:
-                case ZooDefs.OpCode.check:
-                    op.toRequestRecord().serialize(archive, tag);
-                    break;
-                default:
-                    throw new IOException("Invalid type of op");
-            }
-        }
-        new MultiHeader(-1, true, -1).serialize(archive, tag);
-        archive.endRecord(this, tag);
-    }
-
-    @Override
-    public void deserialize(InputArchive archive, String tag) throws IOException {
-        archive.startRecord(tag);
-        MultiHeader h = new MultiHeader();
-        h.deserialize(archive, tag);
-
-        while (!h.getDone()) {
-            switch (h.getType()) {
-                case ZooDefs.OpCode.create:
-                case ZooDefs.OpCode.create2:
-                case ZooDefs.OpCode.createContainer:
-                    CreateRequest cr = new CreateRequest();
-                    cr.deserialize(archive, tag);
-                    add(Op.create(cr.getPath(), cr.getData(), cr.getAcl(), cr.getFlags()));
-                    break;
-                case ZooDefs.OpCode.createTTL:
-                    CreateTTLRequest crTtl = new CreateTTLRequest();
-                    crTtl.deserialize(archive, tag);
-                    add(Op.create(crTtl.getPath(), crTtl.getData(), crTtl.getAcl(), crTtl.getFlags(), crTtl.getTtl()));
-                    break;
-                case ZooDefs.OpCode.delete:
-                    DeleteRequest dr = new DeleteRequest();
-                    dr.deserialize(archive, tag);
-                    add(Op.delete(dr.getPath(), dr.getVersion()));
-                    break;
-                case ZooDefs.OpCode.setData:
-                    SetDataRequest sdr = new SetDataRequest();
-                    sdr.deserialize(archive, tag);
-                    add(Op.setData(sdr.getPath(), sdr.getData(), sdr.getVersion()));
-                    break;
-                case ZooDefs.OpCode.check:
-                    CheckVersionRequest cvr = new CheckVersionRequest();
-                    cvr.deserialize(archive, tag);
-                    add(Op.check(cvr.getPath(), cvr.getVersion()));
-                    break;
-                default:
-                    throw new IOException("Invalid type of op");
-            }
-            h.deserialize(archive, tag);
-        }
-        archive.endRecord(tag);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (!(o instanceof MultiTransactionRecord)) return false;
-
-        MultiTransactionRecord that = (MultiTransactionRecord) o;
-
-        if (ops != null) {
-            Iterator<Op> other = that.ops.iterator();
-            for (Op op : ops) {
-                boolean hasMoreData = other.hasNext();
-                if (!hasMoreData) {
-                    return false;
-                }
-                Op otherOp = other.next();
-                if (!op.equals(otherOp)) {
-                    return false;
-                }
-            }
-            return !other.hasNext();
-        } else {
-            return that.ops == null;
-        }
-
-    }
-
-    @Override
-    public int hashCode() {
-        int h = 1023;
-        for (Op op : ops) {
-            h = h * 25 + op.hashCode();
-        }
-        return h;
-    }
-}
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/Op.java b/zookeeper-server/src/main/java/org/apache/zookeeper/Op.java
index c73cc79..226f64e 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/Op.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/Op.java
@@ -20,10 +20,13 @@ package org.apache.zookeeper;
 import org.apache.jute.Record;
 import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.proto.CheckVersionRequest;
 import org.apache.zookeeper.proto.CreateRequest;
 import org.apache.zookeeper.proto.CreateTTLRequest;
 import org.apache.zookeeper.proto.DeleteRequest;
+import org.apache.zookeeper.proto.GetChildrenRequest;
+import org.apache.zookeeper.proto.GetDataRequest;
 import org.apache.zookeeper.proto.SetDataRequest;
 import org.apache.zookeeper.server.EphemeralType;
 
@@ -32,8 +35,8 @@ import java.util.Iterator;
 import java.util.List;
 
 /**
- * Represents a single operation in a multi-operation transaction.  Each operation can be a create, update
- * or delete or can just be a version check.
+ * Represents a single operation in a multi-operation transaction.  Each operation can be a create, update,
+ * delete, a version check or just read operations like getChildren or getData.
  *
  * Sub-classes of Op each represent each detailed type but should not normally be referenced except via
  * the provided factory methods.
@@ -42,15 +45,21 @@ import java.util.List;
  * @see ZooKeeper#create(String, byte[], java.util.List, CreateMode, org.apache.zookeeper.AsyncCallback.StringCallback, Object)
  * @see ZooKeeper#delete(String, int)
  * @see ZooKeeper#setData(String, byte[], int)
+ * @see ZooKeeper#getData(String, boolean, Stat)
+ * @see ZooKeeper#getChildren(String, boolean)
  */
 public abstract class Op {
+
+    public enum OpKind { TRANSACTION, READ }
     private int type;
     private String path;
+    private OpKind opKind;
 
     // prevent untyped construction
-    private Op(int type, String path) {
+    private Op(int type, String path, OpKind opKind) {
         this.type = type;
         this.path = path;
+        this.opKind = opKind;
     }
 
     /**
@@ -185,6 +194,14 @@ public abstract class Op {
         return new Check(path, version);
     }
 
+    public static Op getChildren(String path) {
+        return new GetChildren(path);
+    }
+
+    public static Op getData(String path) {
+        return new GetData(path);
+    }
+
     /**
      * Gets the integer type code for an Op.  This code should be as from ZooDefs.OpCode
      * @see ZooDefs.OpCode
@@ -203,6 +220,14 @@ public abstract class Op {
     }
 
     /**
+     * Gets the kind of an Op.
+     * @return  The OpKind value.
+     */
+    public OpKind getKind() {
+        return opKind;
+    }
+
+    /**
      * Encodes an op for wire transmission.
      * @return An appropriate Record structure.
      */
@@ -235,7 +260,7 @@ public abstract class Op {
         protected int flags;
 
         private Create(String path, byte[] data, List<ACL> acl, int flags) {
-            super(getOpcode(CreateMode.fromFlag(flags, CreateMode.PERSISTENT)), path);
+            super(getOpcode(CreateMode.fromFlag(flags, CreateMode.PERSISTENT)), path, OpKind.TRANSACTION);
             this.data = data;
             this.acl = acl;
             this.flags = flags;
@@ -249,7 +274,7 @@ public abstract class Op {
         }
 
         private Create(String path, byte[] data, List<ACL> acl, CreateMode createMode) {
-            super(getOpcode(createMode), path);
+            super(getOpcode(createMode), path, OpKind.TRANSACTION);
             this.data = data;
             this.acl = acl;
             this.flags = createMode.toFlag();
@@ -347,7 +372,7 @@ public abstract class Op {
         private int version;
 
         private Delete(String path, int version) {
-            super(ZooDefs.OpCode.delete, path);
+            super(ZooDefs.OpCode.delete, path, OpKind.TRANSACTION);
             this.version = version;
         }
 
@@ -383,7 +408,7 @@ public abstract class Op {
         private int version;
 
         private SetData(String path, byte[] data, int version) {
-            super(ZooDefs.OpCode.setData, path);
+            super(ZooDefs.OpCode.setData, path, OpKind.TRANSACTION);
             this.data = data;
             this.version = version;
         }
@@ -419,7 +444,7 @@ public abstract class Op {
         private int version;
 
         private Check(String path, int version) {
-            super(ZooDefs.OpCode.check, path);
+            super(ZooDefs.OpCode.check, path, OpKind.TRANSACTION);
             this.version = version;
         }
 
@@ -449,4 +474,65 @@ public abstract class Op {
         }
     }
 
+    public static class GetChildren extends Op {
+        GetChildren(String path) {
+            super(ZooDefs.OpCode.getChildren, path, OpKind.READ);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof GetChildren)) return false;
+
+            GetChildren op = (GetChildren) o;
+
+            return getType() == op.getType() && getPath().equals(op.getPath());
+        }
+
+        @Override
+        public int hashCode() {
+            return getType() + getPath().hashCode();
+        }
+
+        @Override
+        public Record toRequestRecord() {
+            return new GetChildrenRequest(getPath(), false);
+        }
+
+        @Override
+        Op withChroot(String path) {
+            return new GetChildren(path);
+        }
+    }
+
+    public static class GetData extends Op {
+        GetData(String path) {
+            super(ZooDefs.OpCode.getData, path, OpKind.READ);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof GetData)) return false;
+
+            GetData op = (GetData) o;
+
+            return getType() == op.getType() && getPath().equals(op.getPath());
+        }
+
+        @Override
+        public int hashCode() {
+            return getType() + getPath().hashCode();
+        }
+
+        @Override
+        public Record toRequestRecord() {
+            return new GetDataRequest(getPath(), false);
+        }
+
+        @Override
+        Op withChroot(String path) {
+            return new GetData(path);
+        }
+    }
 }
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/OpResult.java b/zookeeper-server/src/main/java/org/apache/zookeeper/OpResult.java
index d294b8f..320a8bd 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/OpResult.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/OpResult.java
@@ -20,6 +20,9 @@ package org.apache.zookeeper;
 
 import org.apache.zookeeper.data.Stat;
 
+import java.util.Arrays;
+import java.util.List;
+
 /**
  * Encodes the result of a single part of a multiple operation commit.
  */
@@ -171,6 +174,75 @@ public abstract class OpResult {
     }
 
     /**
+     * A result from a getChildren operation. Provides a list which contains
+     * the names of the children of a given node.
+     */
+    public static class GetChildrenResult extends OpResult {
+        private List<String> children;
+
+        public GetChildrenResult(List<String> children) {
+            super(ZooDefs.OpCode.getChildren);
+            this.children = children;
+        }
+
+        public List<String> getChildren() {
+            return children;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof GetChildrenResult)) return false;
+
+            GetChildrenResult other = (GetChildrenResult) o;
+            return getType() == other.getType() && children.equals(other.children);
+        }
+
+        @Override
+        public int hashCode() {
+            return getType() * 35 + children.hashCode();
+        }
+    }
+
+    /**
+     * A result from a getData operation. The data is represented as a byte array.
+     */
+    public static class GetDataResult extends OpResult {
+
+        private byte[] data;
+        private Stat stat;
+
+        public GetDataResult(byte[] data, Stat stat) {
+            super(ZooDefs.OpCode.getData);
+            this.data = (data == null ? null : Arrays.copyOf(data, data.length));
+            this.stat = stat;
+        }
+
+        public byte[] getData() {
+            return data == null ? null : Arrays.copyOf(data, data.length);
+        }
+        public Stat getStat() {
+            return stat;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof GetDataResult)) return false;
+
+            GetDataResult other = (GetDataResult) o;
+            return getType() == other.getType()
+                    && stat.equals(other.stat)
+                    && Arrays.equals(data, other.data);
+        }
+
+        @Override
+        public int hashCode() {
+            return (int) (getType() * 35 + stat.getMzxid() + Arrays.hashCode(data));
+        }
+    }
+
+    /**
      * An error result from any kind of operation.  The point of error results
      * is that they contain an error code which helps understand what happened.
      * @see KeeperException.Code
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java
index 103eb84..20e0f28 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java
@@ -75,6 +75,8 @@ public class ZooDefs {
 
         public final int createTTL = 21;
 
+        public final int multiRead = 22;
+
         public final int auth = 100;
 
         public final int setWatches = 101;
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
index 8c19304..6f78925 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
@@ -88,6 +88,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.zookeeper.Op.OpKind.READ;
+
 /**
  * This is the main class of ZooKeeper client library. To use a ZooKeeper
  * service, an application must first instantiate an object of ZooKeeper class.
@@ -1795,7 +1797,7 @@ public class ZooKeeper implements AutoCloseable {
     }
 
     /**
-     * Executes multiple ZooKeeper operations or none of them.
+     * Executes multiple ZooKeeper operations. In case of transactions all of them or none of them will be executed.
      * <p>
      * On success, a list of results is returned.
      * On failure, an exception is raised which contains partial results and
@@ -1809,7 +1811,7 @@ public class ZooKeeper implements AutoCloseable {
      * thrown.
      *
      * @param ops An iterable that contains the operations to be done.
-     * These should be created using the factory methods on {@link Op}.
+     * These should be created using the factory methods on {@link Op} and must be the same kind of ops.
      * @return A list of results, one for each input Op, the order of
      * which exactly matches the order of the <code>ops</code> input
      * operations.
@@ -1818,7 +1820,7 @@ public class ZooKeeper implements AutoCloseable {
      * partially succeeded if this exception is thrown.
      * @throws KeeperException If the operation could not be completed
      * due to some error in doing one of the specified ops.
-     * @throws IllegalArgumentException if an invalid path is specified
+     * @throws IllegalArgumentException if an invalid path is specified or different kind of ops are mixed
      *
      * @since 3.4.0
      */
@@ -1874,13 +1876,13 @@ public class ZooKeeper implements AutoCloseable {
         return results;
     }
 
-    private MultiTransactionRecord generateMultiTransaction(Iterable<Op> ops) {
+    private MultiOperationRecord generateMultiTransaction(Iterable<Op> ops) {
         // reconstructing transaction with the chroot prefix
         List<Op> transaction = new ArrayList<Op>();
         for (Op op : ops) {
             transaction.add(withRootPrefix(op));
         }
-        return new MultiTransactionRecord(transaction);
+        return new MultiOperationRecord(transaction);
     }
 
     private Op withRootPrefix(Op op) {
@@ -1893,17 +1895,28 @@ public class ZooKeeper implements AutoCloseable {
         return op;
     }
 
-    protected void multiInternal(MultiTransactionRecord request, MultiCallback cb, Object ctx) {
+    protected void multiInternal(MultiOperationRecord request, MultiCallback cb, Object ctx)
+            throws IllegalArgumentException {
         RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.multi);
+        switch (request.getOpKind()) {
+            case TRANSACTION: h.setType(ZooDefs.OpCode.multi); break;
+            case READ: h.setType(ZooDefs.OpCode.multiRead); break;
+            default:
+                throw new IllegalArgumentException("Unsupported OpKind: " + request.getOpKind());
+        }
         MultiResponse response = new MultiResponse();
         cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, null, null, ctx, null);
     }
 
-    protected List<OpResult> multiInternal(MultiTransactionRecord request)
-        throws InterruptedException, KeeperException {
+    protected List<OpResult> multiInternal(MultiOperationRecord request)
+        throws InterruptedException, KeeperException, IllegalArgumentException {
         RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.multi);
+        switch (request.getOpKind()) {
+            case TRANSACTION: h.setType(ZooDefs.OpCode.multi); break;
+            case READ: h.setType(ZooDefs.OpCode.multiRead); break;
+            default:
+                throw new IllegalArgumentException("Unsupported OpKind: " + request.getOpKind());
+        }
         MultiResponse response = new MultiResponse();
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
@@ -1911,7 +1924,12 @@ public class ZooKeeper implements AutoCloseable {
         }
 
         List<OpResult> results = response.getResultList();
-        
+        // In case of only read operations there is no need to throw an exception
+        // as the subResults are still possibly valid.
+        if (request.getOpKind() == Op.OpKind.READ) {
+            return results;
+        }
+
         ErrorResult fatalError = null;
         for (OpResult result : results) {
             if (result instanceof ErrorResult && ((ErrorResult)result).getErr() != KeeperException.Code.OK.intValue()) {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
index 2267666..81f1145 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
@@ -25,6 +25,8 @@ import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.jute.Record;
+import org.apache.zookeeper.MultiOperationRecord;
+import org.apache.zookeeper.Op;
 import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.data.Id;
 import org.slf4j.Logger;
@@ -38,6 +40,8 @@ import org.apache.zookeeper.OpResult.CheckResult;
 import org.apache.zookeeper.OpResult.CreateResult;
 import org.apache.zookeeper.OpResult.DeleteResult;
 import org.apache.zookeeper.OpResult.ErrorResult;
+import org.apache.zookeeper.OpResult.GetChildrenResult;
+import org.apache.zookeeper.OpResult.GetDataResult;
 import org.apache.zookeeper.OpResult.SetDataResult;
 import org.apache.zookeeper.Watcher.WatcherType;
 import org.apache.zookeeper.ZooDefs;
@@ -270,6 +274,35 @@ public class FinalRequestProcessor implements RequestProcessor {
 
                 break;
             }
+            case OpCode.multiRead: {
+                lastOp = "MLTR";
+                MultiOperationRecord multiReadRecord = new MultiOperationRecord();
+                ByteBufferInputStream.byteBuffer2Record(request.request, multiReadRecord);
+                rsp = new MultiResponse();
+                OpResult subResult;
+                for(Op readOp : multiReadRecord) {
+                    try {
+                        Record rec;
+                        switch(readOp.getType()) {
+                            case OpCode.getChildren:
+                                rec = handleGetChildrenRequest(readOp.toRequestRecord(), cnxn, request.authInfo);
+                                subResult = new GetChildrenResult(((GetChildrenResponse) rec).getChildren());
+                                break;
+                            case OpCode.getData:
+                                rec = handleGetDataRequest(readOp.toRequestRecord(), cnxn, request.authInfo);
+                                GetDataResponse gdr = (GetDataResponse) rec;
+                                subResult = new GetDataResult(gdr.getData(), gdr.getStat());
+                                break;
+                            default:
+                                throw new IOException("Invalid type of readOp");
+                        }
+                    } catch (KeeperException e) {
+                        subResult = new ErrorResult(e.code().intValue());
+                    }
+                    ((MultiResponse)rsp).add(subResult);
+                }
+                break;
+            }
             case OpCode.create: {
                 lastOp = "CREA";
                 rsp = new CreateResponse(rc.path);
@@ -348,17 +381,7 @@ public class FinalRequestProcessor implements RequestProcessor {
                 ByteBufferInputStream.byteBuffer2Record(request.request,
                         getDataRequest);
                 path = getDataRequest.getPath();
-                DataNode n = zks.getZKDatabase().getNode(path);
-                if (n == null) {
-                    throw new KeeperException.NoNodeException();
-                }
-                PrepRequestProcessor.checkACL(zks, request.cnxn, zks.getZKDatabase().aclForNode(n),
-                        ZooDefs.Perms.READ,
-                        request.authInfo, path, null);
-                Stat stat = new Stat();
-                byte b[] = zks.getZKDatabase().getData(path, stat,
-                        getDataRequest.getWatch() ? cnxn : null);
-                rsp = new GetDataResponse(b, stat);
+                rsp = handleGetDataRequest(getDataRequest, cnxn, request.authInfo);
                 break;
             }
             case OpCode.setWatches: {
@@ -417,17 +440,7 @@ public class FinalRequestProcessor implements RequestProcessor {
                 ByteBufferInputStream.byteBuffer2Record(request.request,
                         getChildrenRequest);
                 path = getChildrenRequest.getPath();
-                DataNode n = zks.getZKDatabase().getNode(path);
-                if (n == null) {
-                    throw new KeeperException.NoNodeException();
-                }
-                PrepRequestProcessor.checkACL(zks, request.cnxn, zks.getZKDatabase().aclForNode(n),
-                        ZooDefs.Perms.READ,
-                        request.authInfo, path, null);
-                List<String> children = zks.getZKDatabase().getChildren(
-                        path, null, getChildrenRequest
-                                .getWatch() ? cnxn : null);
-                rsp = new GetChildrenResponse(children);
+                rsp = handleGetChildrenRequest(getChildrenRequest, cnxn, request.authInfo);
                 break;
             }
             case OpCode.getAllChildrenNumber: {
@@ -574,6 +587,37 @@ public class FinalRequestProcessor implements RequestProcessor {
         }
     }
 
+    private Record handleGetChildrenRequest(Record request, ServerCnxn cnxn, List<Id> authInfo)
+            throws KeeperException, IOException {
+        GetChildrenRequest getChildrenRequest = (GetChildrenRequest) request;
+        String path = getChildrenRequest.getPath();
+        DataNode n = zks.getZKDatabase().getNode(path);
+        if (n == null) {
+            throw new KeeperException.NoNodeException();
+        }
+        PrepRequestProcessor.checkACL(zks, cnxn, zks.getZKDatabase().aclForNode(n),
+                ZooDefs.Perms.READ, authInfo, path, null);
+        List<String> children = zks.getZKDatabase().getChildren(path, null,
+                getChildrenRequest.getWatch() ? cnxn : null);
+        return new GetChildrenResponse(children);
+    }
+
+    private Record handleGetDataRequest(Record request, ServerCnxn cnxn, List<Id> authInfo)
+            throws KeeperException, IOException {
+        GetDataRequest getDataRequest = (GetDataRequest) request;
+        String path = getDataRequest.getPath();
+        DataNode n = zks.getZKDatabase().getNode(path);
+        if (n == null) {
+            throw new KeeperException.NoNodeException();
+        }
+        PrepRequestProcessor.checkACL(zks, cnxn, zks.getZKDatabase().aclForNode(n),
+                ZooDefs.Perms.READ, authInfo, path, null);
+        Stat stat = new Stat();
+        byte b[] = zks.getZKDatabase().getData(path, stat,
+                getDataRequest.getWatch() ? cnxn : null);
+        return new GetDataResponse(b, stat);
+    }
+
     private boolean closeSession(ServerCnxnFactory serverCnxnFactory, long sessionId) {
         if (serverCnxnFactory == null) {
             return false;
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
index 8042537..6686736 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
@@ -24,7 +24,7 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.BadArgumentsException;
 import org.apache.zookeeper.KeeperException.Code;
-import org.apache.zookeeper.MultiTransactionRecord;
+import org.apache.zookeeper.MultiOperationRecord;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooDefs.OpCode;
@@ -205,7 +205,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
      * @return a map that contains previously existed records that probably need to be
      *         rolled back in any failure.
      */
-    private Map<String, ChangeRecord> getPendingChanges(MultiTransactionRecord multiRequest) {
+    private Map<String, ChangeRecord> getPendingChanges(MultiOperationRecord multiRequest) {
         Map<String, ChangeRecord> pendingChangeRecords = new HashMap<String, ChangeRecord>();
 
         for (Op op : multiRequest) {
@@ -775,7 +775,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
                 pRequest2Txn(request.type, zks.getNextZxid(), request, checkRequest, true);
                 break;
             case OpCode.multi:
-                MultiTransactionRecord multiRequest = new MultiTransactionRecord();
+                MultiOperationRecord multiRequest = new MultiOperationRecord();
                 try {
                     ByteBufferInputStream.byteBuffer2Record(request.request, multiRequest);
                 } catch(IOException e) {
@@ -868,6 +868,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
             case OpCode.checkWatches:
             case OpCode.removeWatches:
             case OpCode.getEphemerals:
+            case OpCode.multiRead:
                 zks.sessionTracker.checkSession(request.sessionId,
                         request.getOwner());
                 break;
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java
index 6c96792..d730a66 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java
@@ -162,6 +162,7 @@ public class Request {
         case OpCode.getData:
         case OpCode.getEphemerals:
         case OpCode.multi:
+        case OpCode.multiRead:
         case OpCode.ping:
         case OpCode.reconfig:
         case OpCode.setACL:
@@ -185,6 +186,7 @@ public class Request {
         case OpCode.getChildren2:
         case OpCode.getData:
         case OpCode.getEphemerals:
+        case OpCode.multiRead:
             return false;
         case OpCode.create:
         case OpCode.create2:
@@ -233,6 +235,8 @@ public class Request {
             return "check";
         case OpCode.multi:
             return "multi";
+        case OpCode.multiRead:
+            return "multiRead";
         case OpCode.setData:
             return "setData";
         case OpCode.sync:
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java
index 0fbfd2c..d7659f1 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TraceFormatter.java
@@ -89,6 +89,8 @@ public class TraceFormatter {
             return "setWatches";
         case OpCode.sasl:
             return "sasl";
+        case OpCode.multiRead:
+            return "multiRead";
         default:
             return "unknown " + op;
         }
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java
index 0e9559a..acd9181 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.MultiTransactionRecord;
+import org.apache.zookeeper.MultiOperationRecord;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.metrics.MetricsContext;
@@ -72,7 +72,7 @@ public abstract class QuorumZooKeeperServer extends ZooKeeperServer {
         }
 
         if (OpCode.multi == request.type) {
-            MultiTransactionRecord multiTransactionRecord = new MultiTransactionRecord();
+            MultiOperationRecord multiTransactionRecord = new MultiOperationRecord();
             request.request.rewind();
             ByteBufferInputStream.byteBuffer2Record(request.request, multiTransactionRecord);
             request.request.rewind();
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/MultiTransactionRecordTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/MultiOperationRecordTest.java
similarity index 81%
rename from zookeeper-server/src/test/java/org/apache/zookeeper/MultiTransactionRecordTest.java
rename to zookeeper-server/src/test/java/org/apache/zookeeper/MultiOperationRecordTest.java
index d33a3d7..122c1ac 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/MultiTransactionRecordTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/MultiOperationRecordTest.java
@@ -28,16 +28,16 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-public class MultiTransactionRecordTest extends ZKTestCase {
+public class MultiOperationRecordTest extends ZKTestCase {
     @Test
     public void testRoundTrip() throws IOException {
-        MultiTransactionRecord request = new MultiTransactionRecord();
+        MultiOperationRecord request = new MultiOperationRecord();
         request.add(Op.check("check", 1));
         request.add(Op.create("create", "create data".getBytes(), ZooDefs.Ids.CREATOR_ALL_ACL, ZooDefs.Perms.ALL));
         request.add(Op.delete("delete", 17));
         request.add(Op.setData("setData", "set data".getBytes(), 19));
 
-        MultiTransactionRecord decodedRequest = codeDecode(request);
+        MultiOperationRecord decodedRequest = codeDecode(request);
 
         Assert.assertEquals(request, decodedRequest);
         Assert.assertEquals(request.hashCode(), decodedRequest.hashCode());
@@ -45,14 +45,14 @@ public class MultiTransactionRecordTest extends ZKTestCase {
 
     @Test
     public void testEmptyRoundTrip() throws IOException {
-        MultiTransactionRecord request = new MultiTransactionRecord();
-        MultiTransactionRecord decodedRequest = codeDecode(request);
+        MultiOperationRecord request = new MultiOperationRecord();
+        MultiOperationRecord decodedRequest = codeDecode(request);
 
         Assert.assertEquals(request, decodedRequest);
         Assert.assertEquals(request.hashCode(), decodedRequest.hashCode());
     }
 
-    private MultiTransactionRecord codeDecode(MultiTransactionRecord request) throws IOException {
+    private MultiOperationRecord codeDecode(MultiOperationRecord request) throws IOException {
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
         request.serialize(boa, "request");
@@ -61,7 +61,7 @@ public class MultiTransactionRecordTest extends ZKTestCase {
         bb.rewind();
 
         BinaryInputArchive bia = BinaryInputArchive.getArchive(new ByteBufferInputStream(bb));
-        MultiTransactionRecord decodedRequest = new MultiTransactionRecord();
+        MultiOperationRecord decodedRequest = new MultiOperationRecord();
         decodedRequest.deserialize(bia, "request");
         return decodedRequest;
     }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java
index 3995455..04d8df4 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java
@@ -23,7 +23,7 @@ import org.apache.jute.Record;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.SessionExpiredException;
 import org.apache.zookeeper.KeeperException.SessionMovedException;
-import org.apache.zookeeper.MultiTransactionRecord;
+import org.apache.zookeeper.MultiOperationRecord;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZooDefs.Ids;
@@ -116,7 +116,7 @@ public class PrepRequestProcessorTest extends ClientBase {
         pLatch = new CountDownLatch(1);
         processor = new PrepRequestProcessor(zks, new MyRequestProcessor());
 
-        Record record = new MultiTransactionRecord(ops);
+        Record record = new MultiOperationRecord(ops);
         Request req = createRequest(record, OpCode.multi);
 
         processor.pRequest(req);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiTransactionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiOperationTest.java
similarity index 71%
rename from zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiTransactionTest.java
rename to zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiOperationTest.java
index a95fc50..041f077 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiTransactionTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/MultiOperationTest.java
@@ -23,9 +23,14 @@ import static org.junit.Assert.*;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.AsyncCallback.MultiCallback;
@@ -36,6 +41,7 @@ import org.apache.zookeeper.OpResult;
 import org.apache.zookeeper.Transaction;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.OpResult.CheckResult;
 import org.apache.zookeeper.OpResult.CreateResult;
@@ -43,6 +49,8 @@ import org.apache.zookeeper.OpResult.DeleteResult;
 import org.apache.zookeeper.OpResult.ErrorResult;
 import org.apache.zookeeper.OpResult.SetDataResult;
 import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.ZKParameterized;
@@ -57,14 +65,14 @@ import org.slf4j.LoggerFactory;
 
 @RunWith(Parameterized.class)
 @Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class)
-public class MultiTransactionTest extends ClientBase {
-    private static final Logger LOG = LoggerFactory.getLogger(MultiTransactionTest.class);
+public class MultiOperationTest extends ClientBase {
+    private static final Logger LOG = LoggerFactory.getLogger(MultiOperationTest.class);
     private ZooKeeper zk;
     private ZooKeeper zk_chroot;
 
     private final boolean useAsync;
 
-    public MultiTransactionTest(boolean useAsync) {
+    public MultiOperationTest(boolean useAsync) {
         this.useAsync = useAsync;
     }
 
@@ -109,7 +117,8 @@ public class MultiTransactionTest extends ClientBase {
                     res.wait();
                 }
             }
-            if (KeeperException.Code.OK.intValue() != res.rc) {
+            // In case of only OpKind.READ operations, no exception is thrown. Errors only marked in form of ErrorResults.
+            if (KeeperException.Code.OK.intValue() != res.rc && ops.iterator().next().getKind() != Op.OpKind.READ) {
                 KeeperException ke = KeeperException.create(KeeperException.Code.get(res.rc));
                 throw ke;
             }
@@ -787,6 +796,211 @@ public class MultiTransactionTest extends ClientBase {
         assertNull(zk.exists("/t2", false));
     }
 
+    @Test
+    public void testMultiGetChildren() throws Exception {
+        List<String> topLevelNodes = new ArrayList<String>();
+        Map<String, List<String>> childrenNodes = new HashMap<String, List<String>>();
+        // Creating a database where '/fooX' nodes has 'barXY' named children.
+        for (int i = 0; i < 10; i++) {
+            String name = "/foo" + i;
+            zk.create(name, name.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+            topLevelNodes.add(name);
+            childrenNodes.put(name, new ArrayList<>());
+            for (int j = 0; j < 10; j++) {
+                String childname = name + "/bar" + i + j;
+                String childname_s = "bar" + i + j;
+                zk.create(childname, childname.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.EPHEMERAL);
+                childrenNodes.get(name).add(childname_s);
+            }
+        }
+        // Create a multi operation, which queries the children of the nodes in topLevelNodes.
+        List<OpResult> multiChildrenList =
+                multi(zk, topLevelNodes.stream().map(Op::getChildren).collect(Collectors.toList()));
+        for (int i = 0; i < topLevelNodes.size(); i++) {
+            String nodeName = topLevelNodes.get(i);
+            Assert.assertTrue(multiChildrenList.get(i) instanceof OpResult.GetChildrenResult);
+            List<String> childrenList = ((OpResult.GetChildrenResult) multiChildrenList.get(i)).getChildren();
+            // In general, we do not demand an order from the children list but to contain every child.
+            Assert.assertEquals(new TreeSet<String>(childrenList),
+                    new TreeSet<String>(childrenNodes.get(nodeName)));
+
+            List<String> children = zk.getChildren(nodeName, false);
+            Assert.assertEquals(childrenList, children);
+        }
+    }
+
+    @Test
+    public void testMultiGetChildrenSameNode() throws Exception {
+        List<String> childrenNodes = new ArrayList<String>();
+        // Creating a database where '/foo' node has 'barX' named children.
+        String topLevelNode = "/foo";
+        zk.create(topLevelNode, topLevelNode.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT);
+        for (int i = 0; i < 10; i++) {
+            String childname = topLevelNode + "/bar" + i;
+            String childname_s = "bar" + i;
+            zk.create(childname, childname.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.EPHEMERAL);
+            childrenNodes.add(childname_s);
+        }
+
+        // Check for getting the children of the same node twice.
+        List<OpResult> sameChildrenList = multi(zk, Arrays.asList(
+                Op.getChildren(topLevelNode),
+                Op.getChildren(topLevelNode)));
+        // The response should contain two elements which are the same.
+        Assert.assertEquals(sameChildrenList.size(), 2);
+        Assert.assertEquals(sameChildrenList.get(0), sameChildrenList.get(1));
+        // Check the actual result.
+        Assert.assertTrue(sameChildrenList.get(0) instanceof OpResult.GetChildrenResult);
+        OpResult.GetChildrenResult gcr = (OpResult.GetChildrenResult) sameChildrenList.get(0);
+        // In general, we do not demand an order from the children list but to contain every child.
+        Assert.assertEquals(new TreeSet<String>(gcr.getChildren()),
+                new TreeSet<String>(childrenNodes));
+    }
+
+    @Test
+    public void testMultiGetChildrenAuthentication() throws KeeperException, InterruptedException {
+                List<ACL> writeOnly = Collections.singletonList(new ACL(ZooDefs.Perms.WRITE,
+                new Id("world", "anyone")));
+        zk.create("/foo_auth", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/foo_auth/bar", null, Ids.READ_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/foo_no_auth", null, writeOnly, CreateMode.PERSISTENT);
+
+        // Check for normal behaviour.
+        List<OpResult> multiChildrenList = multi(zk, Arrays.asList(Op.getChildren("/foo_auth")));
+        Assert.assertEquals(multiChildrenList.size(), 1);
+        Assert.assertTrue(multiChildrenList.get(0) instanceof  OpResult.GetChildrenResult);
+        List<String> childrenList = ((OpResult.GetChildrenResult) multiChildrenList.get(0)).getChildren();
+        Assert.assertEquals(childrenList.size(), 1);
+        Assert.assertEquals(childrenList.get(0), "bar");
+
+        // Check for authentication violation.
+        multiChildrenList = multi(zk, Arrays.asList(Op.getChildren("/foo_no_auth")));
+
+        Assert.assertEquals(multiChildrenList.size(), 1);
+        Assert.assertTrue(multiChildrenList.get(0) instanceof OpResult.ErrorResult);
+        Assert.assertEquals("Expected NoAuthException for getting the children of a write only node",
+                ((OpResult.ErrorResult) multiChildrenList.get(0)).getErr(), KeeperException.Code.NOAUTH.intValue());
+
+    }
+
+    @Test
+    public void testMultiGetChildrenMixedAuthenticationErrorFirst() throws KeeperException, InterruptedException {
+        List<ACL> writeOnly = Collections.singletonList(new ACL(ZooDefs.Perms.WRITE,
+                new Id("world", "anyone")));
+        zk.create("/foo_auth", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/foo_auth/bar", null, Ids.READ_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/foo_no_auth", null, writeOnly, CreateMode.PERSISTENT);
+        List<OpResult> multiChildrenList;
+
+        // Mixed nodes, the operation after the error should return RuntimeInconsistency error.
+        multiChildrenList = multi(zk, Arrays.asList(Op.getChildren("/foo_no_auth"), Op.getChildren("/foo_auth")));
+
+        Assert.assertEquals(multiChildrenList.size(), 2);
+        Assert.assertTrue(multiChildrenList.get(0) instanceof OpResult.ErrorResult);
+        Assert.assertEquals("Expected NoAuthException for getting the children of a write only node",
+                ((OpResult.ErrorResult) multiChildrenList.get(0)).getErr(), KeeperException.Code.NOAUTH.intValue());
+
+        Assert.assertTrue(multiChildrenList.get(1) instanceof OpResult.GetChildrenResult);
+        List<String> childrenList = ((OpResult.GetChildrenResult) multiChildrenList.get(1)).getChildren();
+        Assert.assertEquals(childrenList.size(), 1);
+        Assert.assertEquals(childrenList.get(0), "bar");
+    }
+
+    @Test
+    public void testMultiGetChildrenMixedAuthenticationCorrectFirst() throws KeeperException, InterruptedException {
+        List<ACL> writeOnly = Collections.singletonList(new ACL(ZooDefs.Perms.WRITE,
+                new Id("world", "anyone")));
+        zk.create("/foo_auth", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/foo_auth/bar", null, Ids.READ_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/foo_no_auth", null, writeOnly, CreateMode.PERSISTENT);
+
+        // Check for getting the children of the nodes with mixed authentication.
+        // The getChildren operation returns GetChildrenResult if it happened before the error.
+        List<OpResult> multiChildrenList;
+        multiChildrenList = multi(zk, Arrays.asList(Op.getChildren("/foo_auth"), Op.getChildren("/foo_no_auth")));
+        Assert.assertSame(multiChildrenList.size(), 2);
+
+        Assert.assertTrue(multiChildrenList.get(0) instanceof OpResult.GetChildrenResult);
+        List<String> childrenList = ((OpResult.GetChildrenResult) multiChildrenList.get(0)).getChildren();
+        Assert.assertEquals(childrenList.size(), 1);
+        Assert.assertEquals(childrenList.get(0), "bar");
+
+        Assert.assertTrue(multiChildrenList.get(1) instanceof OpResult.ErrorResult);
+        Assert.assertEquals("Expected NoAuthException for getting the children of a write only node",
+                ((OpResult.ErrorResult) multiChildrenList.get(1)).getErr(), KeeperException.Code.NOAUTH.intValue());
+    }
+
+    @Test
+    public void testMultiGetData() throws Exception {
+        zk.create("/node1", "data1".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/node2", "data2".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+        List<OpResult> multiData = multi(zk, Arrays.asList(Op.getData("/node1"), Op.getData("/node2")));
+        Assert.assertEquals(multiData.size(), 2);
+        Assert.assertArrayEquals(((OpResult.GetDataResult) multiData.get(0)).getData(),"data1".getBytes());
+        Assert.assertArrayEquals(((OpResult.GetDataResult) multiData.get(1)).getData(),"data2".getBytes());
+    }
+
+    @Test
+    public void testMultiRead() throws Exception {
+        zk.create("/node1", "data1".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/node2", "data2".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
+        zk.create("/node1/node1", "data11".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.create("/node1/node2", "data12".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+        List<OpResult> multiRead = multi(zk, Arrays.asList(Op.getChildren("/node1"), Op.getData("/node1"),
+                                                           Op.getChildren("/node2"), Op.getData("/node2")));
+        Assert.assertEquals(multiRead.size(), 4);
+        Assert.assertTrue(multiRead.get(0) instanceof OpResult.GetChildrenResult);
+        List<String> childrenList = ((OpResult.GetChildrenResult) multiRead.get(0)).getChildren();
+        Assert.assertEquals(childrenList.size(), 2);
+        Assert.assertEquals(new TreeSet<String>(childrenList), new TreeSet<String>(Arrays.asList("node1", "node2")));
+
+        Assert.assertArrayEquals(((OpResult.GetDataResult) multiRead.get(1)).getData(),"data1".getBytes());
+        Stat stat = ((OpResult.GetDataResult) multiRead.get(1)).getStat();
+        Assert.assertEquals(stat.getMzxid(), stat.getCzxid());
+        Assert.assertEquals(stat.getCtime(), stat.getMtime());
+        Assert.assertEquals(2, stat.getCversion());
+        Assert.assertEquals(0, stat.getVersion());
+        Assert.assertEquals(0, stat.getAversion());
+        Assert.assertEquals(0, stat.getEphemeralOwner());
+        Assert.assertEquals(5, stat.getDataLength());
+        Assert.assertEquals(2, stat.getNumChildren());
+
+        Assert.assertTrue(multiRead.get(2) instanceof OpResult.GetChildrenResult);
+        childrenList = ((OpResult.GetChildrenResult) multiRead.get(2)).getChildren();
+        Assert.assertTrue(childrenList.isEmpty());
+
+        Assert.assertArrayEquals(((OpResult.GetDataResult) multiRead.get(3)).getData(),"data2".getBytes());
+        stat = ((OpResult.GetDataResult) multiRead.get(3)).getStat();
+        Assert.assertEquals(stat.getMzxid(), stat.getCzxid());
+        Assert.assertEquals(stat.getMzxid(), stat.getPzxid());
+        Assert.assertEquals(stat.getCtime(), stat.getMtime());
+        Assert.assertEquals(0, stat.getCversion());
+        Assert.assertEquals(0, stat.getVersion());
+        Assert.assertEquals(0, stat.getAversion());
+        Assert.assertEquals(zk.getSessionId(), stat.getEphemeralOwner());
+        Assert.assertEquals(5, stat.getDataLength());
+        Assert.assertEquals(0, stat.getNumChildren());
+    }
+
+
+    @Test
+    public void testMixedReadAndTransaction() throws Exception {
+        zk.create("/node", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        try {
+        List<OpResult> multiRead = multi(zk, Arrays.asList(Op.setData("/node1", "data1".getBytes(), -1),
+                                                           Op.getData("/node1")));
+        fail("Mixed kind of operations are not allowed");
+        } catch (IllegalArgumentException e) {
+            // expected
+        }
+    }
+
     private static class HasTriggeredWatcher implements Watcher {
         private final CountDownLatch triggered = new CountDownLatch(1);