You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by el...@apache.org on 2016/03/24 05:26:08 UTC

[1/4] calcite git commit: [CALCITE-1128] Implement JDBC batch update methods in remote driver

Repository: calcite
Updated Branches:
  refs/heads/master 857c06b67 -> 5dfa3f1ec


http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index 228ba8d..e6c2fed 100644
--- a/avatica/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/avatica/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -24,6 +24,7 @@ import org.apache.calcite.avatica.remote.LocalService;
 import org.apache.calcite.avatica.remote.ProtobufTranslation;
 import org.apache.calcite.avatica.remote.ProtobufTranslationImpl;
 import org.apache.calcite.avatica.remote.Service;
+import org.apache.calcite.avatica.remote.TypedValue;
 
 import com.google.common.cache.Cache;
 
@@ -35,6 +36,8 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.lang.reflect.Field;
 import java.sql.Connection;
@@ -49,11 +52,14 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.sql.Types;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Calendar;
 import java.util.List;
 import java.util.Map;
 import java.util.TimeZone;
+import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
@@ -61,8 +67,10 @@ import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -74,6 +82,8 @@ import static org.junit.Assert.fail;
 @RunWith(Parameterized.class)
 @NotThreadSafe // for testConnectionIsolation
 public class RemoteDriverTest {
+  private static final Logger LOG = LoggerFactory.getLogger(RemoteDriverTest.class);
+
   public static final String LJS =
       LocalJdbcServiceFactory.class.getName();
 
@@ -1024,6 +1034,394 @@ public class RemoteDriverTest {
         equalTo(Connection.TRANSACTION_READ_COMMITTED));
   }
 
+  @Test public void testBatchExecute() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              executeBatchUpdate(c1);
+            }
+          }, getLocalConnection());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  private void executeBatchUpdate(Connection conn) throws Exception {
+    final int numRows = 10;
+    try (Statement stmt = conn.createStatement()) {
+      final String tableName = AvaticaUtils.unique("BATCH_EXECUTE");
+      LOG.info("Creating table {}", tableName);
+      final String createCommand = String.format("create table if not exists %s ("
+          + "id int not null, "
+          + "msg varchar(10) not null)", tableName);
+      assertFalse("Failed to create table", stmt.execute(createCommand));
+
+      final String updatePrefix = String.format("INSERT INTO %s values(", tableName);
+      for (int i = 0; i < numRows;  i++) {
+        stmt.addBatch(updatePrefix + i + ", '" + Integer.toString(i) + "')");
+      }
+
+      int[] updateCounts = stmt.executeBatch();
+      assertEquals("Unexpected number of update counts returned", numRows, updateCounts.length);
+      for (int i = 0; i < updateCounts.length; i++) {
+        assertEquals("Unexpected update count at index " + i, 1, updateCounts[i]);
+      }
+
+      ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName + " ORDER BY id asc");
+      assertNotNull("ResultSet was null", rs);
+      for (int i = 0; i < numRows; i++) {
+        assertTrue("ResultSet should have a result", rs.next());
+        assertEquals("Wrong integer value for row " + i, i, rs.getInt(1));
+        assertEquals("Wrong string value for row " + i, Integer.toString(i), rs.getString(2));
+      }
+      assertFalse("ResultSet should have no more records", rs.next());
+    }
+  }
+
+  @Test public void testPreparedBatches() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              executePreparedBatchUpdate(c1);
+            }
+          }, getLocalConnection());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  private void executePreparedBatchUpdate(Connection conn) throws Exception {
+    final int numRows = 10;
+    final String tableName = AvaticaUtils.unique("PREPARED_BATCH_EXECUTE");
+    LOG.info("Creating table {}", tableName);
+    try (Statement stmt = conn.createStatement()) {
+      final String createCommand = String.format("create table if not exists %s ("
+          + "id int not null, "
+          + "msg varchar(10) not null)", tableName);
+      assertFalse("Failed to create table", stmt.execute(createCommand));
+    }
+
+    final String insertSql = String.format("INSERT INTO %s values(?, ?)", tableName);
+    try (PreparedStatement pstmt = conn.prepareStatement(insertSql)) {
+      // Add batches with the prepared statement
+      for (int i = 0; i < numRows; i++) {
+        pstmt.setInt(1, i);
+        pstmt.setString(2, Integer.toString(i));
+        pstmt.addBatch();
+      }
+
+      int[] updateCounts = pstmt.executeBatch();
+      assertEquals("Unexpected number of update counts returned", numRows, updateCounts.length);
+      for (int i = 0; i < updateCounts.length; i++) {
+        assertEquals("Unexpected update count at index " + i, 1, updateCounts[i]);
+      }
+    }
+
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName + " ORDER BY id asc");
+      assertNotNull("ResultSet was null", rs);
+      for (int i = 0; i < numRows; i++) {
+        assertTrue("ResultSet should have a result", rs.next());
+        assertEquals("Wrong integer value for row " + i, i, rs.getInt(1));
+        assertEquals("Wrong string value for row " + i, Integer.toString(i), rs.getString(2));
+      }
+      assertFalse("ResultSet should have no more records", rs.next());
+    }
+  }
+
+  @Test public void testPreparedInsert() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              executePreparedInsert(c1);
+            }
+          }, getLocalConnection());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  private void executePreparedInsert(Connection conn) throws Exception {
+    final int numRows = 10;
+    final String tableName = AvaticaUtils.unique("PREPARED_INSERT_EXECUTE");
+    LOG.info("Creating table {}", tableName);
+    try (Statement stmt = conn.createStatement()) {
+      final String createCommand = String.format("create table if not exists %s ("
+          + "id int not null, "
+          + "msg varchar(10) not null)", tableName);
+      assertFalse("Failed to create table", stmt.execute(createCommand));
+    }
+
+    final String insertSql = String.format("INSERT INTO %s values(?, ?)", tableName);
+    try (PreparedStatement pstmt = conn.prepareStatement(insertSql)) {
+      // Add batches with the prepared statement
+      for (int i = 0; i < numRows; i++) {
+        pstmt.setInt(1, i);
+        pstmt.setString(2, Integer.toString(i));
+        assertEquals(1, pstmt.executeUpdate());
+      }
+    }
+
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName + " ORDER BY id asc");
+      assertNotNull("ResultSet was null", rs);
+      for (int i = 0; i < numRows; i++) {
+        assertTrue("ResultSet should have a result", rs.next());
+        assertEquals("Wrong integer value for row " + i, i, rs.getInt(1));
+        assertEquals("Wrong string value for row " + i, Integer.toString(i), rs.getString(2));
+      }
+      assertFalse("ResultSet should have no more records", rs.next());
+    }
+  }
+
+  @Test public void testPreparedInsertWithNulls() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              executePreparedInsertWithNulls(c1);
+            }
+          }, getLocalConnection());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  private void executePreparedInsertWithNulls(Connection conn) throws Exception {
+    final int numRows = 10;
+    final String tableName = AvaticaUtils.unique("PREPARED_INSERT_EXECUTE_NULLS");
+    LOG.info("Creating table {}", tableName);
+    try (Statement stmt = conn.createStatement()) {
+      final String createCommand = String.format("create table if not exists %s ("
+          + "id int not null, "
+          + "msg varchar(10))", tableName);
+      assertFalse("Failed to create table", stmt.execute(createCommand));
+    }
+
+    final String insertSql = String.format("INSERT INTO %s values(?, ?)", tableName);
+    try (PreparedStatement pstmt = conn.prepareStatement(insertSql)) {
+      // Add batches with the prepared statement
+      for (int i = 0; i < numRows; i++) {
+        pstmt.setInt(1, i);
+        // Even inserts are non-null, odd are null
+        if (0 == i % 2) {
+          pstmt.setString(2, Integer.toString(i));
+        } else {
+          pstmt.setNull(2, Types.VARCHAR);
+        }
+        assertEquals(1, pstmt.executeUpdate());
+      }
+    }
+
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName + " ORDER BY id asc");
+      assertNotNull("ResultSet was null", rs);
+      for (int i = 0; i < numRows; i++) {
+        assertTrue("ResultSet should have a result", rs.next());
+        assertEquals("Wrong integer value for row " + i, i, rs.getInt(1));
+        if (0 == i % 2) {
+          assertEquals("Wrong string value for row " + i, Integer.toString(i), rs.getString(2));
+        } else {
+          assertNull("Expected null value for row " + i, rs.getString(2));
+        }
+      }
+      assertFalse("ResultSet should have no more records", rs.next());
+    }
+  }
+
+  @Test public void testBatchInsertWithNulls() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              executeBatchInsertWithNulls(c1);
+            }
+          }, getLocalConnection());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  private void executeBatchInsertWithNulls(Connection conn) throws Exception {
+    final int numRows = 10;
+    final String tableName = AvaticaUtils.unique("BATCH_INSERT_EXECUTE_NULLS");
+    LOG.info("Creating table {}", tableName);
+    try (Statement stmt = conn.createStatement()) {
+      final String createCommand = String.format("create table if not exists %s ("
+          + "id int not null, "
+          + "msg varchar(10))", tableName);
+      assertFalse("Failed to create table", stmt.execute(createCommand));
+    }
+
+    final String insertSql = String.format("INSERT INTO %s values(?, ?)", tableName);
+    try (PreparedStatement pstmt = conn.prepareStatement(insertSql)) {
+      // Add batches with the prepared statement
+      for (int i = 0; i < numRows; i++) {
+        pstmt.setInt(1, i);
+        // Even inserts are non-null, odd are null
+        if (0 == i % 2) {
+          pstmt.setString(2, Integer.toString(i));
+        } else {
+          pstmt.setNull(2, Types.VARCHAR);
+        }
+        pstmt.addBatch();
+      }
+      // Verify that all updates were successful
+      int[] updateCounts = pstmt.executeBatch();
+      assertEquals(numRows, updateCounts.length);
+      int[] expectedCounts = new int[numRows];
+      Arrays.fill(expectedCounts, 1);
+      assertArrayEquals(expectedCounts, updateCounts);
+    }
+
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName + " ORDER BY id asc");
+      assertNotNull("ResultSet was null", rs);
+      for (int i = 0; i < numRows; i++) {
+        assertTrue("ResultSet should have a result", rs.next());
+        assertEquals("Wrong integer value for row " + i, i, rs.getInt(1));
+        if (0 == i % 2) {
+          assertEquals("Wrong string value for row " + i, Integer.toString(i), rs.getString(2));
+        } else {
+          assertNull("Expected null value for row " + i, rs.getString(2));
+        }
+      }
+      assertFalse("ResultSet should have no more records", rs.next());
+    }
+  }
+
+  @Test public void preparedStatementParameterCopies() throws Exception {
+    // When implementing the JDBC batch APIs, it's important that we are copying the
+    // TypedValues and caching them in the AvaticaPreparedStatement. Otherwise, when we submit
+    // the batch, the parameter values for the last update added will be reflected in all previous
+    // updates added to the batch.
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      final String tableName = AvaticaUtils.unique("PREPAREDSTATEMENT_VALUES");
+      final Connection conn = getLocalConnection();
+      try (Statement stmt = conn.createStatement()) {
+        final String sql = "CREATE TABLE " + tableName
+            + " (id varchar(1) not null, col1 varchar(1) not null)";
+        assertFalse(stmt.execute(sql));
+      }
+      try (final PreparedStatement pstmt =
+          conn.prepareStatement("INSERT INTO " + tableName + " values(?, ?)")) {
+        pstmt.setString(1, "a");
+        pstmt.setString(2, "b");
+
+        @SuppressWarnings("resource")
+        AvaticaPreparedStatement apstmt = (AvaticaPreparedStatement) pstmt;
+        TypedValue[] slots = apstmt.slots;
+
+        assertEquals("Unexpected number of values", 2, slots.length);
+
+        List<TypedValue> valuesReference = apstmt.getParameterValues();
+        assertEquals(2, valuesReference.size());
+        assertEquals(slots[0], valuesReference.get(0));
+        assertEquals(slots[1], valuesReference.get(1));
+        List<TypedValue> copiedValues = apstmt.copyParameterValues();
+        assertEquals(2, valuesReference.size());
+        assertEquals(slots[0], copiedValues.get(0));
+        assertEquals(slots[1], copiedValues.get(1));
+
+        slots[0] = null;
+        slots[1] = null;
+
+        // Modifications to the array are reflected in the List from getParameterValues()
+        assertNull(valuesReference.get(0));
+        assertNull(valuesReference.get(1));
+        // copyParameterValues() copied the underlying array, so updates to slots is not reflected
+        assertNotNull(copiedValues.get(0));
+        assertNotNull(copiedValues.get(1));
+      }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testBatchInsertWithDates() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              executeBatchInsertWithDates(c1);
+            }
+          }, getLocalConnection());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  private void executeBatchInsertWithDates(Connection conn) throws Exception {
+    final Calendar calendar = Calendar.getInstance();
+    long now = calendar.getTime().getTime();
+    final int numRows = 10;
+    final String tableName = AvaticaUtils.unique("BATCH_INSERT_EXECUTE_DATES");
+    LOG.info("Creating table {}", tableName);
+    try (Statement stmt = conn.createStatement()) {
+      final String dropCommand = String.format("drop table if exists %s", tableName);
+      assertFalse("Failed to drop table", stmt.execute(dropCommand));
+      final String createCommand = String.format("create table %s ("
+          + "id char(15) not null, "
+          + "created_date date not null, "
+          + "val_string varchar)", tableName);
+      assertFalse("Failed to create table", stmt.execute(createCommand));
+    }
+
+    final String insertSql = String.format("INSERT INTO %s values(?, ?, ?)", tableName);
+    try (PreparedStatement pstmt = conn.prepareStatement(insertSql)) {
+      // Add batches with the prepared statement
+      for (int i = 0; i < numRows; i++) {
+        pstmt.setString(1, Integer.toString(i));
+        pstmt.setDate(2, new Date(now + i), calendar);
+        pstmt.setString(3, UUID.randomUUID().toString());
+        pstmt.addBatch();
+      }
+      // Verify that all updates were successful
+      int[] updateCounts = pstmt.executeBatch();
+      assertEquals(numRows, updateCounts.length);
+      int[] expectedCounts = new int[numRows];
+      Arrays.fill(expectedCounts, 1);
+      assertArrayEquals(expectedCounts, updateCounts);
+    }
+
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName + " ORDER BY id asc");
+      assertNotNull("ResultSet was null", rs);
+      for (int i = 0; i < numRows; i++) {
+        assertTrue("ResultSet should have a result", rs.next());
+        assertEquals("Wrong value for row " + i, Integer.toString(i), rs.getString(1).trim());
+
+        Date actual = rs.getDate(2);
+        calendar.setTime(actual);
+        int actualDay = calendar.get(Calendar.DAY_OF_MONTH);
+        int actualMonth = calendar.get(Calendar.MONTH);
+        int actualYear = calendar.get(Calendar.YEAR);
+
+        Date expected = new Date(now + i);
+        calendar.setTime(expected);
+        int expectedDay = calendar.get(Calendar.DAY_OF_MONTH);
+        int expectedMonth = calendar.get(Calendar.MONTH);
+        int expectedYear = calendar.get(Calendar.YEAR);
+        assertEquals("Wrong day for row " + i, expectedDay, actualDay);
+        assertEquals("Wrong month for row " + i, expectedMonth, actualMonth);
+        assertEquals("Wrong year for row " + i, expectedYear, actualYear);
+
+        assertNotNull("Non-null string for row " + i, rs.getString(3));
+      }
+      assertFalse("ResultSet should have no more records", rs.next());
+    }
+  }
+
   /**
    * Factory that creates a service based on a local JDBC connection.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/site/_docs/json_reference.md
----------------------------------------------------------------------
diff --git a/avatica/site/_docs/json_reference.md b/avatica/site/_docs/json_reference.md
index 85db8f5..f0f8f65 100644
--- a/avatica/site/_docs/json_reference.md
+++ b/avatica/site/_docs/json_reference.md
@@ -13,8 +13,10 @@ requests:
   - { name: "CreateStatementRequest" }
   - { name: "DatabasePropertyRequest" }
   - { name: "ExecuteRequest" }
+  - { name: "ExecuteBatchRequest" }
   - { name: "FetchRequest" }
   - { name: "OpenConnectionRequest" }
+  - { name: "PrepareAndExecuteBatchRequest" }
   - { name: "PrepareAndExecuteRequest" }
   - { name: "PrepareRequest" }
   - { name: "RollbackRequest" }
@@ -49,6 +51,7 @@ responses:
   - { name: "CreateStatementResponse" }
   - { name: "DatabasePropertyResponse" }
   - { name: "ErrorResponse" }
+  - { name: "ExecuteBatchResponse" }
   - { name: "ExecuteResponse" }
   - { name: "FetchResponse" }
   - { name: "OpenConnectionResponse" }
@@ -231,6 +234,26 @@ This request is used to fetch all <a href="#databaseproperty">database propertie
 
 `connectionId` (required string) The identifier of the connection to use when fetching the database properties.
 
+### ExecuteBatchRequest
+
+This request is used to execute a batch of updates on a PreparedStatement.
+
+{% highlight json %}
+{
+  "request": "executeBatch",
+  "connectionId": "000000-0000-0000-00000000",
+  "statementId": 12345,
+  "parameterValues": [ [ TypedValue, TypedValue, ... ], [ TypedValue, TypedValue, ...], ... ]
+}
+{% endhighlight %}
+
+`connectionId` (required string) The identifier of the connection to use when fetching the database properties.
+
+`statementId` (required integer) The identifier of the statement created using the above connection.
+
+`parameterValues` (required array of array) An array of arrays of <a href="#typedvalue">TypedValue</a>'s. Each element
+  in the array is an update to a row, while the outer array represents the entire "batch" of updates.
+
 ### ExecuteRequest
 
 This request is used to execute a PreparedStatement, optionally with values to bind to the parameters in the Statement.
@@ -288,6 +311,25 @@ This request is used to open a new Connection in the Avatica server.
 
 `info` (optional string-to-string map) A Map containing properties to include when creating the Connection.
 
+### PrepareAndExecuteBatchRequest
+
+This request is used as short-hand to create a Statement and execute an batch of SQL commands in that Statement.
+
+{% highlight json %}
+{
+  "request": "prepareAndExecuteBatch",
+  "connectionId": "000000-0000-0000-00000000",
+  "statementId": 12345,
+  "sqlCommands", [ "SQL Command", "SQL Command", ... ]
+}
+{% endhighlight %}
+
+`connectionId` (required string) The identifier for the connection to use.
+
+`statementId` (required integer) The identifier for the statement created by the above connection to use.
+
+`sqlCommands` (required array of strings) An array of SQL commands
+
 ### PrepareAndExecuteRequest
 
 This request is used as a short-hand for create a Statement and fetching the first batch of results in a single call without any parameter substitution.
@@ -560,6 +602,33 @@ A response when an error was caught executing a request. Any request may return
 
 `rpcMetadata` <a href="#rpcmetadata">Server metadata</a> about this call.
 
+### ExecuteBatchResponse
+
+A response to <a href="#executebatchrequest">ExecuteBatchRequest</a> and <a href="#prepareandexecutebatchrequest">PrepareAndExecuteRequest</a>
+which encapsulates the update counts for a batch of updates.
+
+{% highlight json %}
+{
+  "response": "executeBatch",
+  "connectionId": "000000-0000-0000-00000000",
+  "statementId": 12345,
+  "updateCounts": [ 1, 1, 0, 1, ... ],
+  "missingStatement": false,
+  "rpcMetadata": RpcMetadata
+}
+{% endhighlight %}
+
+`connectionId` The identifier for the connection used to create the statement.
+
+`statementId` The identifier for the created statement.
+
+`updateCounts` An array of integers corresponding to each update contained in the batch that was executed.
+
+`missingStatement` True if the operation failed because the Statement is not cached in the server, false otherwise.
+
+`rpcMetadata` <a href="#rpcmetadata">Server metadata</a> about this call.
+
+
 ### ExecuteResponse
 
 A response to the <a href="#executerequest">ExecuteRequest</a> which contains the results for a metadata query.

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/site/_docs/protobuf_reference.md
----------------------------------------------------------------------
diff --git a/avatica/site/_docs/protobuf_reference.md b/avatica/site/_docs/protobuf_reference.md
index 4840ceb..9a76c03 100644
--- a/avatica/site/_docs/protobuf_reference.md
+++ b/avatica/site/_docs/protobuf_reference.md
@@ -12,9 +12,11 @@ requests:
   - { name: "ConnectionSyncRequest" }
   - { name: "CreateStatementRequest" }
   - { name: "DatabasePropertyRequest" }
+  - { name: "ExecuteBatchRequest" }
   - { name: "ExecuteRequest" }
   - { name: "FetchRequest" }
   - { name: "OpenConnectionRequest" }
+  - { name: "PrepareAndExecuteBatchRequest" }
   - { name: "PrepareAndExecuteRequest" }
   - { name: "PrepareRequest" }
   - { name: "RollbackRequest" }
@@ -43,6 +45,7 @@ miscellaneous:
   - { name: "StatementType" }
   - { name: "Style" }
   - { name: "TypedValue" }
+  - { name: "UpdateBatch" }
   - { name: "WireMessage" }
 responses:
   - { name: "CloseConnectionResponse" }
@@ -52,6 +55,7 @@ responses:
   - { name: "CreateStatementResponse" }
   - { name: "DatabasePropertyResponse" }
   - { name: "ErrorResponse" }
+  - { name: "ExecuteBatchResponse" }
   - { name: "ExecuteResponse" }
   - { name: "FetchResponse" }
   - { name: "OpenConnectionResponse" }
@@ -234,6 +238,24 @@ message DatabasePropertyRequest {
 
 `connection_id` The identifier of the connection to use when fetching the database properties.
 
+### ExecuteBatchRequest
+
+This request is used to execute a batch of updates against a PreparedStatement.
+
+{% highlight protobuf %}
+message ExecuteBatchRequest {
+  string connection_id = 1;
+  uint32 statement_id = 2;
+  repeated UpdateBatch updates = 3;
+}
+{% endhighlight %}
+
+`connection_id` A string which refers to a connection.
+
+`statement_id` An integer which refers to a statement.
+
+`updates` A list of <a href="#updatebatch">UpdateBatch</a>'s; the batch of updates.
+
 ### ExecuteRequest
 
 This request is used to execute a PreparedStatement, optionally with values to bind to the parameters in the Statement.
@@ -291,6 +313,24 @@ message OpenConnectionRequest {
 
 `info` A Map containing properties to include when creating the Connection.
 
+### PrepareAndExecuteBatchRequest
+
+This request is used as short-hand to create a Statement and execute a batch of updates against that Statement.
+
+{% highlight protobuf %}
+message PrepareAndExecuteBatchRequest {
+  string connection_id = 1;
+  uint32 statement_id = 2;
+  repeated string sql_commands = 3;
+}
+{% endhighlight %}
+
+`connection_id` The identifier for the connection to use.
+
+`statement_id` The identifier for the statement created by the above connection to use.
+
+`sql_commands` A list of SQL commands to execute; a batch.
+
 ### PrepareAndExecuteRequest
 
 This request is used as a short-hand for create a Statement and fetching the first batch of results in a single call without any parameter substitution.
@@ -554,6 +594,30 @@ message ErrorResponse {
 
 `metadata` <a href="#rpcmetadata">Server metadata</a> about this call.
 
+### ExecuteBatchResponse
+
+A response to the <a href="#executebatchrequest">ExecuteBatchRequest</a> and <a href="#prepareandexecutebatchrequest">PrepareAndExecuteBatchRequest</a>.
+
+{% highlight protobuf %}
+message ExecuteBatchResponse {
+  string connection_id = 1;
+  uint32 statement_id = 2;
+  repeated uint32 update_counts = 3;
+  bool missing_statement = 4;
+  RpcMetadata metadata = 5;
+}
+{% endhighlight %}
+
+`connection_id` The ID referring to the connection that was used.
+
+`statment_id` The ID referring to the statement that was used.
+
+`update_counts` An array of integer values corresponding to the update count for each update in the batch.
+
+`missing_statement` A boolean which denotes if the request failed due to a missing statement.
+
+`metadata` <a href="#rpcmetadata">Server metadata</a> about this call.
+
 ### ExecuteResponse
 
 A response to the <a href="#executerequest">ExecuteRequest</a> which contains the results for a metadata query.
@@ -1165,6 +1229,18 @@ message TypedValue {
 
 `null` A boolean which denotes if the value was null.
 
+### UpdateBatch
+
+This is a message which serves as a wrapper around a collection of <a href="#typedvalue">TypedValue</a>'s.
+
+{% highlight protobuf %}
+message UpdateBatch {
+  repeated TypedValue parameter_values = 1;
+}
+{% endhighlight %}
+
+`parameter_values` A collection of parameter values for one SQL command update.
+
 ### WireMessage
 
 This message wraps all `Request`s and `Response`s.


[4/4] calcite git commit: [CALCITE-1128] Implement JDBC batch update methods in remote driver

Posted by el...@apache.org.
[CALCITE-1128] Implement JDBC batch update methods in remote driver

This commit provides an implementation for:

* Statement.addBatch(String)
* PreparedStatement.addBatch()
* PreparedStatement.executeBatch()

The implementation is fairly straightforward except for the addition
of a new server interface: ProtobufMeta. This is a new interface which
the Meta implementation can choose to also implement to provide a "native"
implementation on top of Protobuf objects instead of the Avatica POJOs.

During the investigations Avatica performance pre-1.7.0, it was found
that converting protobufs to POJOs was a very hot code path. This short-circuit
helps us avoid extra objects on the heap and computation to create them
in what should be a very hot code path for write-workloads.

Closes apache/calcite#209


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/5dfa3f1e
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/5dfa3f1e
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/5dfa3f1e

Branch: refs/heads/master
Commit: 5dfa3f1ece54d2f95057c5b5097dc0f7fae693ee
Parents: 857c06b
Author: Josh Elser <el...@apache.org>
Authored: Thu Mar 3 17:25:56 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Mar 24 00:25:32 2016 -0400

----------------------------------------------------------------------
 .../calcite/avatica/AvaticaConnection.java      |   25 +
 .../avatica/AvaticaPreparedStatement.java       |   37 +-
 .../calcite/avatica/AvaticaStatement.java       |   33 +-
 .../java/org/apache/calcite/avatica/Meta.java   |   29 +
 .../apache/calcite/avatica/proto/Requests.java  | 2418 +++++++++++++++++-
 .../apache/calcite/avatica/proto/Responses.java |  970 ++++++-
 .../calcite/avatica/remote/JsonService.java     |   16 +
 .../calcite/avatica/remote/LocalService.java    |   32 +
 .../calcite/avatica/remote/ProtobufMeta.java    |   45 +
 .../calcite/avatica/remote/ProtobufService.java |    8 +
 .../avatica/remote/ProtobufTranslationImpl.java |   15 +
 .../calcite/avatica/remote/RemoteMeta.java      |   23 +
 .../apache/calcite/avatica/remote/Service.java  |  276 +-
 .../calcite/avatica/remote/TypedValue.java      |  145 +-
 avatica/core/src/main/protobuf/requests.proto   |   18 +
 avatica/core/src/main/protobuf/responses.proto  |    9 +
 .../avatica/remote/ExecuteBatchRequestTest.java |   79 +
 .../remote/ProtobufTranslationImplTest.java     |   11 +
 .../calcite/avatica/test/JsonHandlerTest.java   |    8 +
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |   82 +-
 .../calcite/avatica/RemoteDriverTest.java       |  398 +++
 avatica/site/_docs/json_reference.md            |   69 +
 avatica/site/_docs/protobuf_reference.md        |   76 +
 23 files changed, 4775 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
index 2d89f45..091fe6b 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.avatica;
 
+import org.apache.calcite.avatica.Meta.ExecuteBatchResult;
 import org.apache.calcite.avatica.Meta.MetaResultSet;
 import org.apache.calcite.avatica.remote.Service.ErrorResponse;
 import org.apache.calcite.avatica.remote.Service.OpenConnectionRequest;
@@ -505,6 +506,25 @@ public abstract class AvaticaConnection implements Connection {
     return statement.openResultSet;
   }
 
+  /** Executes a batch update using an {@link AvaticaPreparedStatement}.
+   *
+   * @param pstmt The prepared statement.
+   * @return An array of update counts containing one element for each command in the batch.
+   */
+  protected int[] executeBatchUpdateInternal(AvaticaPreparedStatement pstmt) throws SQLException {
+    try {
+      // Get the handle from the statement
+      Meta.StatementHandle handle = pstmt.handle;
+      // Execute it against meta
+      final Meta.ExecuteBatchResult executeBatchResult =
+          meta.executeBatch(handle, pstmt.getParameterValueBatch());
+      // Send back just the update counts
+      return executeBatchResult.updateCounts;
+    } catch (Exception e) {
+      throw helper.createException(e.getMessage(), e);
+    }
+  }
+
   /** Returns whether a a statement is capable of updates and if so,
    * and the statement's {@code updateCount} is still -1, proceeds to
    * get updateCount value from statement's resultSet.
@@ -581,6 +601,11 @@ public abstract class AvaticaConnection implements Connection {
     return meta.prepareAndExecute(statement.handle, sql, maxRowCount, callback);
   }
 
+  protected ExecuteBatchResult prepareAndUpdateBatch(final AvaticaStatement statement,
+      final List<String> queries) throws NoSuchStatementException, SQLException {
+    return meta.prepareAndExecuteBatch(statement.handle, queries);
+  }
+
   protected ResultSet createResultSet(Meta.MetaResultSet metaResultSet, QueryState state)
       throws SQLException {
     final Meta.StatementHandle h = new Meta.StatementHandle(

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
index f3a950a..9b8a292 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
@@ -35,6 +35,7 @@ import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Calendar;
 import java.util.List;
@@ -52,6 +53,7 @@ public abstract class AvaticaPreparedStatement
   private final ResultSetMetaData resultSetMetaData;
   private Calendar calendar;
   protected final TypedValue[] slots;
+  protected final List<List<TypedValue>> parameterValueBatch;
 
   /**
    * Creates an AvaticaPreparedStatement.
@@ -75,12 +77,27 @@ public abstract class AvaticaPreparedStatement
     this.slots = new TypedValue[signature.parameters.size()];
     this.resultSetMetaData =
         connection.factory.newResultSetMetaData(this, signature);
+    this.parameterValueBatch = new ArrayList<>();
   }
 
   @Override protected List<TypedValue> getParameterValues() {
     return Arrays.asList(slots);
   }
 
+  /** Returns a copy of the current parameter values.
+   * @return A copied list of the parameter values
+   */
+  protected List<TypedValue> copyParameterValues() {
+    // For implementing batch update, we need to make a copy of slots, not just a thin reference
+    // to it as as list. Otherwise, subsequent setFoo(..) calls will alter the underlying array
+    // and modify our cached TypedValue list.
+    List<TypedValue> copy = new ArrayList<>(slots.length);
+    for (TypedValue value : slots) {
+      copy.add(value);
+    }
+    return copy;
+  }
+
   /** Returns a calendar in the connection's time zone, creating one the first
    * time this method is called.
    *
@@ -103,6 +120,10 @@ public abstract class AvaticaPreparedStatement
     return calendar;
   }
 
+  protected List<List<TypedValue>> getParameterValueBatch() {
+    return this.parameterValueBatch;
+  }
+
   // implement PreparedStatement
 
   public ResultSet executeQuery() throws SQLException {
@@ -213,7 +234,21 @@ public abstract class AvaticaPreparedStatement
   }
 
   public void addBatch() throws SQLException {
-    throw connection.helper.unsupported();
+    // Need to copy the parameterValues into a new list, not wrap the array in a list
+    // as getParameterValues does.
+    this.parameterValueBatch.add(copyParameterValues());
+  }
+
+  @Override public int[] executeBatch() throws SQLException {
+    // Overriding the implementation in AvaticaStatement.
+    try {
+      final int[] updateCounts = getConnection().executeBatchUpdateInternal(this);
+      return updateCounts;
+    } finally {
+      // If we failed to send this batch, that's a problem for the user to handle, not us.
+      // Make sure we always clear the statements we collected to submit in one RPC.
+      this.parameterValueBatch.clear();
+    }
   }
 
   public void setCharacterStream(int parameterIndex, Reader reader, int length)

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
index cfd1d45..a58fc15 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
@@ -24,6 +24,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
@@ -67,6 +68,8 @@ public abstract class AvaticaStatement
 
   private Meta.Signature signature;
 
+  private final List<String> batchedSql;
+
   protected void setSignature(Meta.Signature signature) {
     this.signature = signature;
   }
@@ -109,6 +112,7 @@ public abstract class AvaticaStatement
     }
     connection.statementMap.put(h.id, this);
     this.handle = h;
+    this.batchedSql = new ArrayList<>();
   }
 
   /** Returns the identifier of the statement, unique within its connection. */
@@ -148,6 +152,25 @@ public abstract class AvaticaStatement
         + connection.maxRetriesPerExecute + " attempts.");
   }
 
+  /**
+   * Executes a collection of updates in a single batch RPC.
+   *
+   * @return an array of integers mapping to the update count per SQL command.
+   */
+  protected int[] executeBatchInternal() throws SQLException {
+    for (int i = 0; i < connection.maxRetriesPerExecute; i++) {
+      try {
+        Meta.ExecuteBatchResult result = connection.prepareAndUpdateBatch(this, batchedSql);
+        return result.updateCounts;
+      } catch (NoSuchStatementException e) {
+        resetStatement();
+      }
+    }
+
+    throw new RuntimeException("Failed to successfully execute batch update after "
+        +  connection.maxRetriesPerExecute + " attempts");
+  }
+
   protected void resetStatement() {
     // Invalidate the old statement
     connection.statementMap.remove(handle.id);
@@ -359,7 +382,7 @@ public abstract class AvaticaStatement
   }
 
   public void addBatch(String sql) throws SQLException {
-    throw connection.helper.unsupported();
+    this.batchedSql.add(Objects.requireNonNull(sql));
   }
 
   public void clearBatch() throws SQLException {
@@ -367,7 +390,13 @@ public abstract class AvaticaStatement
   }
 
   public int[] executeBatch() throws SQLException {
-    throw connection.helper.unsupported();
+    try {
+      return executeBatchInternal();
+    } finally {
+      // If we failed to send this batch, that's a problem for the user to handle, not us.
+      // Make sure we always clear the statements we collected to submit in one RPC.
+      this.batchedSql.clear();
+    }
   }
 
   public AvaticaConnection getConnection() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java b/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
index 4cc460c..41ca0ee 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -228,6 +228,24 @@ public interface Meta {
   ExecuteResult prepareAndExecute(StatementHandle h, String sql,
       long maxRowCount, PrepareCallback callback) throws NoSuchStatementException;
 
+  /** Prepares a statement and then executes a number of SQL commands in one pass.
+   *
+   * @param h Statement handle
+   * @param sqlCommands SQL commands to run
+   * @return An array of update counts containing one element for each command in the batch.
+   */
+  ExecuteBatchResult prepareAndExecuteBatch(StatementHandle h, List<String> sqlCommands)
+      throws NoSuchStatementException;
+
+  /** Executes a collection of bound parameter values on a prepared statement.
+   *
+   * @param h Statement handle
+   * @param parameterValues A collection of list of typed values, one list per batch
+   * @return An array of update counts containing one element for each command in the batch.
+   */
+  ExecuteBatchResult executeBatch(StatementHandle h, List<List<TypedValue>> parameterValues)
+      throws NoSuchStatementException;
+
   /** Returns a frame of rows.
    *
    * <p>The frame describes whether there may be another frame. If there is not
@@ -424,6 +442,17 @@ public interface Meta {
     }
   }
 
+  /**
+   * Response from a collection of SQL commands or parameter values in a single batch.
+   */
+  class ExecuteBatchResult {
+    public final int[] updateCounts;
+
+    public ExecuteBatchResult(int[] updateCounts) {
+      this.updateCounts = Objects.requireNonNull(updateCounts);
+    }
+  }
+
   /** Meta data from which a result set can be constructed.
    *
    * <p>If {@code updateCount} is not -1, the result is just a count. A result


[3/4] calcite git commit: [CALCITE-1128] Implement JDBC batch update methods in remote driver

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Requests.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Requests.java b/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Requests.java
index 6d34757..3fcbe69 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Requests.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Requests.java
@@ -12035,6 +12035,2381 @@ package org.apache.calcite.avatica.proto;
 
   }
 
+  public interface PrepareAndExecuteBatchRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:PrepareAndExecuteBatchRequest)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    java.lang.String getConnectionId();
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getConnectionIdBytes();
+
+    /**
+     * <code>optional uint32 statement_id = 2;</code>
+     */
+    int getStatementId();
+
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    com.google.protobuf.ProtocolStringList
+        getSqlCommandsList();
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    int getSqlCommandsCount();
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    java.lang.String getSqlCommands(int index);
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    com.google.protobuf.ByteString
+        getSqlCommandsBytes(int index);
+  }
+  /**
+   * Protobuf type {@code PrepareAndExecuteBatchRequest}
+   *
+   * <pre>
+   * Request to prepare and execute a collection of sql statements.
+   * </pre>
+   */
+  public  static final class PrepareAndExecuteBatchRequest extends
+      com.google.protobuf.GeneratedMessage implements
+      // @@protoc_insertion_point(message_implements:PrepareAndExecuteBatchRequest)
+      PrepareAndExecuteBatchRequestOrBuilder {
+    // Use PrepareAndExecuteBatchRequest.newBuilder() to construct.
+    private PrepareAndExecuteBatchRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+    }
+    private PrepareAndExecuteBatchRequest() {
+      connectionId_ = "";
+      statementId_ = 0;
+      sqlCommands_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private PrepareAndExecuteBatchRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry) {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              String s = input.readStringRequireUtf8();
+
+              connectionId_ = s;
+              break;
+            }
+            case 16: {
+
+              statementId_ = input.readUInt32();
+              break;
+            }
+            case 26: {
+              String s = input.readStringRequireUtf8();
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                sqlCommands_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              sqlCommands_.add(s);
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw new RuntimeException(e.setUnfinishedMessage(this));
+      } catch (java.io.IOException e) {
+        throw new RuntimeException(
+            new com.google.protobuf.InvalidProtocolBufferException(
+                e.getMessage()).setUnfinishedMessage(this));
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          sqlCommands_ = sqlCommands_.getUnmodifiableView();
+        }
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.calcite.avatica.proto.Requests.internal_static_PrepareAndExecuteBatchRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.calcite.avatica.proto.Requests.internal_static_PrepareAndExecuteBatchRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest.class, org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int CONNECTION_ID_FIELD_NUMBER = 1;
+    private volatile java.lang.Object connectionId_;
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    public java.lang.String getConnectionId() {
+      java.lang.Object ref = connectionId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        connectionId_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getConnectionIdBytes() {
+      java.lang.Object ref = connectionId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        connectionId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int STATEMENT_ID_FIELD_NUMBER = 2;
+    private int statementId_;
+    /**
+     * <code>optional uint32 statement_id = 2;</code>
+     */
+    public int getStatementId() {
+      return statementId_;
+    }
+
+    public static final int SQL_COMMANDS_FIELD_NUMBER = 3;
+    private com.google.protobuf.LazyStringList sqlCommands_;
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    public com.google.protobuf.ProtocolStringList
+        getSqlCommandsList() {
+      return sqlCommands_;
+    }
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    public int getSqlCommandsCount() {
+      return sqlCommands_.size();
+    }
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    public java.lang.String getSqlCommands(int index) {
+      return sqlCommands_.get(index);
+    }
+    /**
+     * <code>repeated string sql_commands = 3;</code>
+     */
+    public com.google.protobuf.ByteString
+        getSqlCommandsBytes(int index) {
+      return sqlCommands_.getByteString(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (!getConnectionIdBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessage.writeString(output, 1, connectionId_);
+      }
+      if (statementId_ != 0) {
+        output.writeUInt32(2, statementId_);
+      }
+      for (int i = 0; i < sqlCommands_.size(); i++) {
+        com.google.protobuf.GeneratedMessage.writeString(output, 3, sqlCommands_.getRaw(i));
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (!getConnectionIdBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessage.computeStringSize(1, connectionId_);
+      }
+      if (statementId_ != 0) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(2, statementId_);
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < sqlCommands_.size(); i++) {
+          dataSize += computeStringSizeNoTag(sqlCommands_.getRaw(i));
+        }
+        size += dataSize;
+        size += 1 * getSqlCommandsList().size();
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code PrepareAndExecuteBatchRequest}
+     *
+     * <pre>
+     * Request to prepare and execute a collection of sql statements.
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:PrepareAndExecuteBatchRequest)
+        org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_PrepareAndExecuteBatchRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_PrepareAndExecuteBatchRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest.class, org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest.Builder.class);
+      }
+
+      // Construct using org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        connectionId_ = "";
+
+        statementId_ = 0;
+
+        sqlCommands_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_PrepareAndExecuteBatchRequest_descriptor;
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest getDefaultInstanceForType() {
+        return org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest.getDefaultInstance();
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest build() {
+        org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest buildPartial() {
+        org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest result = new org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        result.connectionId_ = connectionId_;
+        result.statementId_ = statementId_;
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          sqlCommands_ = sqlCommands_.getUnmodifiableView();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        }
+        result.sqlCommands_ = sqlCommands_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest) {
+          return mergeFrom((org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest other) {
+        if (other == org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest.getDefaultInstance()) return this;
+        if (!other.getConnectionId().isEmpty()) {
+          connectionId_ = other.connectionId_;
+          onChanged();
+        }
+        if (other.getStatementId() != 0) {
+          setStatementId(other.getStatementId());
+        }
+        if (!other.sqlCommands_.isEmpty()) {
+          if (sqlCommands_.isEmpty()) {
+            sqlCommands_ = other.sqlCommands_;
+            bitField0_ = (bitField0_ & ~0x00000004);
+          } else {
+            ensureSqlCommandsIsMutable();
+            sqlCommands_.addAll(other.sqlCommands_);
+          }
+          onChanged();
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.lang.Object connectionId_ = "";
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public java.lang.String getConnectionId() {
+        java.lang.Object ref = connectionId_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          connectionId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getConnectionIdBytes() {
+        java.lang.Object ref = connectionId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          connectionId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder setConnectionId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        connectionId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder clearConnectionId() {
+        
+        connectionId_ = getDefaultInstance().getConnectionId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder setConnectionIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        connectionId_ = value;
+        onChanged();
+        return this;
+      }
+
+      private int statementId_ ;
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public int getStatementId() {
+        return statementId_;
+      }
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public Builder setStatementId(int value) {
+        
+        statementId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public Builder clearStatementId() {
+        
+        statementId_ = 0;
+        onChanged();
+        return this;
+      }
+
+      private com.google.protobuf.LazyStringList sqlCommands_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureSqlCommandsIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          sqlCommands_ = new com.google.protobuf.LazyStringArrayList(sqlCommands_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public com.google.protobuf.ProtocolStringList
+          getSqlCommandsList() {
+        return sqlCommands_.getUnmodifiableView();
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public int getSqlCommandsCount() {
+        return sqlCommands_.size();
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public java.lang.String getSqlCommands(int index) {
+        return sqlCommands_.get(index);
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public com.google.protobuf.ByteString
+          getSqlCommandsBytes(int index) {
+        return sqlCommands_.getByteString(index);
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public Builder setSqlCommands(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureSqlCommandsIsMutable();
+        sqlCommands_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public Builder addSqlCommands(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureSqlCommandsIsMutable();
+        sqlCommands_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public Builder addAllSqlCommands(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureSqlCommandsIsMutable();
+        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            values, sqlCommands_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public Builder clearSqlCommands() {
+        sqlCommands_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string sql_commands = 3;</code>
+       */
+      public Builder addSqlCommandsBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        ensureSqlCommandsIsMutable();
+        sqlCommands_.add(value);
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:PrepareAndExecuteBatchRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:PrepareAndExecuteBatchRequest)
+    private static final org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest();
+    }
+
+    public static org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<PrepareAndExecuteBatchRequest>
+        PARSER = new com.google.protobuf.AbstractParser<PrepareAndExecuteBatchRequest>() {
+      public PrepareAndExecuteBatchRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        try {
+          return new PrepareAndExecuteBatchRequest(input, extensionRegistry);
+        } catch (RuntimeException e) {
+          if (e.getCause() instanceof
+              com.google.protobuf.InvalidProtocolBufferException) {
+            throw (com.google.protobuf.InvalidProtocolBufferException)
+                e.getCause();
+          }
+          throw e;
+        }
+      }
+    };
+
+    public static com.google.protobuf.Parser<PrepareAndExecuteBatchRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<PrepareAndExecuteBatchRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface UpdateBatchOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:UpdateBatch)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> 
+        getParameterValuesList();
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValue getParameterValues(int index);
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    int getParameterValuesCount();
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+        getParameterValuesOrBuilderList();
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getParameterValuesOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code UpdateBatch}
+   *
+   * <pre>
+   * Each command is a list of TypedValues
+   * </pre>
+   */
+  public  static final class UpdateBatch extends
+      com.google.protobuf.GeneratedMessage implements
+      // @@protoc_insertion_point(message_implements:UpdateBatch)
+      UpdateBatchOrBuilder {
+    // Use UpdateBatch.newBuilder() to construct.
+    private UpdateBatch(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+    }
+    private UpdateBatch() {
+      parameterValues_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private UpdateBatch(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry) {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                parameterValues_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Common.TypedValue>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              parameterValues_.add(input.readMessage(org.apache.calcite.avatica.proto.Common.TypedValue.parser(), extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw new RuntimeException(e.setUnfinishedMessage(this));
+      } catch (java.io.IOException e) {
+        throw new RuntimeException(
+            new com.google.protobuf.InvalidProtocolBufferException(
+                e.getMessage()).setUnfinishedMessage(this));
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          parameterValues_ = java.util.Collections.unmodifiableList(parameterValues_);
+        }
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.calcite.avatica.proto.Requests.internal_static_UpdateBatch_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.calcite.avatica.proto.Requests.internal_static_UpdateBatch_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.calcite.avatica.proto.Requests.UpdateBatch.class, org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder.class);
+    }
+
+    public static final int PARAMETER_VALUES_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> parameterValues_;
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getParameterValuesList() {
+      return parameterValues_;
+    }
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+        getParameterValuesOrBuilderList() {
+      return parameterValues_;
+    }
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    public int getParameterValuesCount() {
+      return parameterValues_.size();
+    }
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValue getParameterValues(int index) {
+      return parameterValues_.get(index);
+    }
+    /**
+     * <code>repeated .TypedValue parameter_values = 1;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getParameterValuesOrBuilder(
+        int index) {
+      return parameterValues_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      for (int i = 0; i < parameterValues_.size(); i++) {
+        output.writeMessage(1, parameterValues_.get(i));
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < parameterValues_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, parameterValues_.get(i));
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.calcite.avatica.proto.Requests.UpdateBatch prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code UpdateBatch}
+     *
+     * <pre>
+     * Each command is a list of TypedValues
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:UpdateBatch)
+        org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_UpdateBatch_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_UpdateBatch_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.calcite.avatica.proto.Requests.UpdateBatch.class, org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder.class);
+      }
+
+      // Construct using org.apache.calcite.avatica.proto.Requests.UpdateBatch.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getParameterValuesFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (parameterValuesBuilder_ == null) {
+          parameterValues_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          parameterValuesBuilder_.clear();
+        }
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_UpdateBatch_descriptor;
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatch getDefaultInstanceForType() {
+        return org.apache.calcite.avatica.proto.Requests.UpdateBatch.getDefaultInstance();
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatch build() {
+        org.apache.calcite.avatica.proto.Requests.UpdateBatch result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatch buildPartial() {
+        org.apache.calcite.avatica.proto.Requests.UpdateBatch result = new org.apache.calcite.avatica.proto.Requests.UpdateBatch(this);
+        int from_bitField0_ = bitField0_;
+        if (parameterValuesBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            parameterValues_ = java.util.Collections.unmodifiableList(parameterValues_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.parameterValues_ = parameterValues_;
+        } else {
+          result.parameterValues_ = parameterValuesBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.calcite.avatica.proto.Requests.UpdateBatch) {
+          return mergeFrom((org.apache.calcite.avatica.proto.Requests.UpdateBatch)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.calcite.avatica.proto.Requests.UpdateBatch other) {
+        if (other == org.apache.calcite.avatica.proto.Requests.UpdateBatch.getDefaultInstance()) return this;
+        if (parameterValuesBuilder_ == null) {
+          if (!other.parameterValues_.isEmpty()) {
+            if (parameterValues_.isEmpty()) {
+              parameterValues_ = other.parameterValues_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureParameterValuesIsMutable();
+              parameterValues_.addAll(other.parameterValues_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.parameterValues_.isEmpty()) {
+            if (parameterValuesBuilder_.isEmpty()) {
+              parameterValuesBuilder_.dispose();
+              parameterValuesBuilder_ = null;
+              parameterValues_ = other.parameterValues_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              parameterValuesBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getParameterValuesFieldBuilder() : null;
+            } else {
+              parameterValuesBuilder_.addAllMessages(other.parameterValues_);
+            }
+          }
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.calcite.avatica.proto.Requests.UpdateBatch parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.calcite.avatica.proto.Requests.UpdateBatch) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> parameterValues_ =
+        java.util.Collections.emptyList();
+      private void ensureParameterValuesIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          parameterValues_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Common.TypedValue>(parameterValues_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> parameterValuesBuilder_;
+
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getParameterValuesList() {
+        if (parameterValuesBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(parameterValues_);
+        } else {
+          return parameterValuesBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public int getParameterValuesCount() {
+        if (parameterValuesBuilder_ == null) {
+          return parameterValues_.size();
+        } else {
+          return parameterValuesBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue getParameterValues(int index) {
+        if (parameterValuesBuilder_ == null) {
+          return parameterValues_.get(index);
+        } else {
+          return parameterValuesBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder setParameterValues(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (parameterValuesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureParameterValuesIsMutable();
+          parameterValues_.set(index, value);
+          onChanged();
+        } else {
+          parameterValuesBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder setParameterValues(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (parameterValuesBuilder_ == null) {
+          ensureParameterValuesIsMutable();
+          parameterValues_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          parameterValuesBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder addParameterValues(org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (parameterValuesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureParameterValuesIsMutable();
+          parameterValues_.add(value);
+          onChanged();
+        } else {
+          parameterValuesBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder addParameterValues(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (parameterValuesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureParameterValuesIsMutable();
+          parameterValues_.add(index, value);
+          onChanged();
+        } else {
+          parameterValuesBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder addParameterValues(
+          org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (parameterValuesBuilder_ == null) {
+          ensureParameterValuesIsMutable();
+          parameterValues_.add(builderForValue.build());
+          onChanged();
+        } else {
+          parameterValuesBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder addParameterValues(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (parameterValuesBuilder_ == null) {
+          ensureParameterValuesIsMutable();
+          parameterValues_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          parameterValuesBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder addAllParameterValues(
+          java.lang.Iterable<? extends org.apache.calcite.avatica.proto.Common.TypedValue> values) {
+        if (parameterValuesBuilder_ == null) {
+          ensureParameterValuesIsMutable();
+          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, parameterValues_);
+          onChanged();
+        } else {
+          parameterValuesBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder clearParameterValues() {
+        if (parameterValuesBuilder_ == null) {
+          parameterValues_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          parameterValuesBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public Builder removeParameterValues(int index) {
+        if (parameterValuesBuilder_ == null) {
+          ensureParameterValuesIsMutable();
+          parameterValues_.remove(index);
+          onChanged();
+        } else {
+          parameterValuesBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder getParameterValuesBuilder(
+          int index) {
+        return getParameterValuesFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getParameterValuesOrBuilder(
+          int index) {
+        if (parameterValuesBuilder_ == null) {
+          return parameterValues_.get(index);  } else {
+          return parameterValuesBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+           getParameterValuesOrBuilderList() {
+        if (parameterValuesBuilder_ != null) {
+          return parameterValuesBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(parameterValues_);
+        }
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addParameterValuesBuilder() {
+        return getParameterValuesFieldBuilder().addBuilder(
+            org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addParameterValuesBuilder(
+          int index) {
+        return getParameterValuesFieldBuilder().addBuilder(
+            index, org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .TypedValue parameter_values = 1;</code>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue.Builder> 
+           getParameterValuesBuilderList() {
+        return getParameterValuesFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+          getParameterValuesFieldBuilder() {
+        if (parameterValuesBuilder_ == null) {
+          parameterValuesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder>(
+                  parameterValues_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          parameterValues_ = null;
+        }
+        return parameterValuesBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:UpdateBatch)
+    }
+
+    // @@protoc_insertion_point(class_scope:UpdateBatch)
+    private static final org.apache.calcite.avatica.proto.Requests.UpdateBatch DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.calcite.avatica.proto.Requests.UpdateBatch();
+    }
+
+    public static org.apache.calcite.avatica.proto.Requests.UpdateBatch getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<UpdateBatch>
+        PARSER = new com.google.protobuf.AbstractParser<UpdateBatch>() {
+      public UpdateBatch parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        try {
+          return new UpdateBatch(input, extensionRegistry);
+        } catch (RuntimeException e) {
+          if (e.getCause() instanceof
+              com.google.protobuf.InvalidProtocolBufferException) {
+            throw (com.google.protobuf.InvalidProtocolBufferException)
+                e.getCause();
+          }
+          throw e;
+        }
+      }
+    };
+
+    public static com.google.protobuf.Parser<UpdateBatch> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<UpdateBatch> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.calcite.avatica.proto.Requests.UpdateBatch getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface ExecuteBatchRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:ExecuteBatchRequest)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    java.lang.String getConnectionId();
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getConnectionIdBytes();
+
+    /**
+     * <code>optional uint32 statement_id = 2;</code>
+     */
+    int getStatementId();
+
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    java.util.List<org.apache.calcite.avatica.proto.Requests.UpdateBatch> 
+        getUpdatesList();
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    org.apache.calcite.avatica.proto.Requests.UpdateBatch getUpdates(int index);
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    int getUpdatesCount();
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    java.util.List<? extends org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder> 
+        getUpdatesOrBuilderList();
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder getUpdatesOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code ExecuteBatchRequest}
+   */
+  public  static final class ExecuteBatchRequest extends
+      com.google.protobuf.GeneratedMessage implements
+      // @@protoc_insertion_point(message_implements:ExecuteBatchRequest)
+      ExecuteBatchRequestOrBuilder {
+    // Use ExecuteBatchRequest.newBuilder() to construct.
+    private ExecuteBatchRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+    }
+    private ExecuteBatchRequest() {
+      connectionId_ = "";
+      statementId_ = 0;
+      updates_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private ExecuteBatchRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry) {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              String s = input.readStringRequireUtf8();
+
+              connectionId_ = s;
+              break;
+            }
+            case 16: {
+
+              statementId_ = input.readUInt32();
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                updates_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Requests.UpdateBatch>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              updates_.add(input.readMessage(org.apache.calcite.avatica.proto.Requests.UpdateBatch.parser(), extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw new RuntimeException(e.setUnfinishedMessage(this));
+      } catch (java.io.IOException e) {
+        throw new RuntimeException(
+            new com.google.protobuf.InvalidProtocolBufferException(
+                e.getMessage()).setUnfinishedMessage(this));
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          updates_ = java.util.Collections.unmodifiableList(updates_);
+        }
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.calcite.avatica.proto.Requests.internal_static_ExecuteBatchRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.calcite.avatica.proto.Requests.internal_static_ExecuteBatchRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest.class, org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int CONNECTION_ID_FIELD_NUMBER = 1;
+    private volatile java.lang.Object connectionId_;
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    public java.lang.String getConnectionId() {
+      java.lang.Object ref = connectionId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        connectionId_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getConnectionIdBytes() {
+      java.lang.Object ref = connectionId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        connectionId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int STATEMENT_ID_FIELD_NUMBER = 2;
+    private int statementId_;
+    /**
+     * <code>optional uint32 statement_id = 2;</code>
+     */
+    public int getStatementId() {
+      return statementId_;
+    }
+
+    public static final int UPDATES_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.calcite.avatica.proto.Requests.UpdateBatch> updates_;
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    public java.util.List<org.apache.calcite.avatica.proto.Requests.UpdateBatch> getUpdatesList() {
+      return updates_;
+    }
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    public java.util.List<? extends org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder> 
+        getUpdatesOrBuilderList() {
+      return updates_;
+    }
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    public int getUpdatesCount() {
+      return updates_.size();
+    }
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    public org.apache.calcite.avatica.proto.Requests.UpdateBatch getUpdates(int index) {
+      return updates_.get(index);
+    }
+    /**
+     * <code>repeated .UpdateBatch updates = 3;</code>
+     *
+     * <pre>
+     * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+     * </pre>
+     */
+    public org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder getUpdatesOrBuilder(
+        int index) {
+      return updates_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (!getConnectionIdBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessage.writeString(output, 1, connectionId_);
+      }
+      if (statementId_ != 0) {
+        output.writeUInt32(2, statementId_);
+      }
+      for (int i = 0; i < updates_.size(); i++) {
+        output.writeMessage(3, updates_.get(i));
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (!getConnectionIdBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessage.computeStringSize(1, connectionId_);
+      }
+      if (statementId_ != 0) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(2, statementId_);
+      }
+      for (int i = 0; i < updates_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, updates_.get(i));
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ExecuteBatchRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:ExecuteBatchRequest)
+        org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_ExecuteBatchRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_ExecuteBatchRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest.class, org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest.Builder.class);
+      }
+
+      // Construct using org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUpdatesFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        connectionId_ = "";
+
+        statementId_ = 0;
+
+        if (updatesBuilder_ == null) {
+          updates_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          updatesBuilder_.clear();
+        }
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.calcite.avatica.proto.Requests.internal_static_ExecuteBatchRequest_descriptor;
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest getDefaultInstanceForType() {
+        return org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest.getDefaultInstance();
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest build() {
+        org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest buildPartial() {
+        org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest result = new org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        result.connectionId_ = connectionId_;
+        result.statementId_ = statementId_;
+        if (updatesBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            updates_ = java.util.Collections.unmodifiableList(updates_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.updates_ = updates_;
+        } else {
+          result.updates_ = updatesBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest) {
+          return mergeFrom((org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest other) {
+        if (other == org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest.getDefaultInstance()) return this;
+        if (!other.getConnectionId().isEmpty()) {
+          connectionId_ = other.connectionId_;
+          onChanged();
+        }
+        if (other.getStatementId() != 0) {
+          setStatementId(other.getStatementId());
+        }
+        if (updatesBuilder_ == null) {
+          if (!other.updates_.isEmpty()) {
+            if (updates_.isEmpty()) {
+              updates_ = other.updates_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureUpdatesIsMutable();
+              updates_.addAll(other.updates_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.updates_.isEmpty()) {
+            if (updatesBuilder_.isEmpty()) {
+              updatesBuilder_.dispose();
+              updatesBuilder_ = null;
+              updates_ = other.updates_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              updatesBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getUpdatesFieldBuilder() : null;
+            } else {
+              updatesBuilder_.addAllMessages(other.updates_);
+            }
+          }
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.lang.Object connectionId_ = "";
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public java.lang.String getConnectionId() {
+        java.lang.Object ref = connectionId_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          connectionId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getConnectionIdBytes() {
+        java.lang.Object ref = connectionId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          connectionId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder setConnectionId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        connectionId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder clearConnectionId() {
+        
+        connectionId_ = getDefaultInstance().getConnectionId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder setConnectionIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        connectionId_ = value;
+        onChanged();
+        return this;
+      }
+
+      private int statementId_ ;
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public int getStatementId() {
+        return statementId_;
+      }
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public Builder setStatementId(int value) {
+        
+        statementId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public Builder clearStatementId() {
+        
+        statementId_ = 0;
+        onChanged();
+        return this;
+      }
+
+      private java.util.List<org.apache.calcite.avatica.proto.Requests.UpdateBatch> updates_ =
+        java.util.Collections.emptyList();
+      private void ensureUpdatesIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          updates_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Requests.UpdateBatch>(updates_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.calcite.avatica.proto.Requests.UpdateBatch, org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder, org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder> updatesBuilder_;
+
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Requests.UpdateBatch> getUpdatesList() {
+        if (updatesBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(updates_);
+        } else {
+          return updatesBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public int getUpdatesCount() {
+        if (updatesBuilder_ == null) {
+          return updates_.size();
+        } else {
+          return updatesBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatch getUpdates(int index) {
+        if (updatesBuilder_ == null) {
+          return updates_.get(index);
+        } else {
+          return updatesBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder setUpdates(
+          int index, org.apache.calcite.avatica.proto.Requests.UpdateBatch value) {
+        if (updatesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureUpdatesIsMutable();
+          updates_.set(index, value);
+          onChanged();
+        } else {
+          updatesBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder setUpdates(
+          int index, org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder builderForValue) {
+        if (updatesBuilder_ == null) {
+          ensureUpdatesIsMutable();
+          updates_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          updatesBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder addUpdates(org.apache.calcite.avatica.proto.Requests.UpdateBatch value) {
+        if (updatesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureUpdatesIsMutable();
+          updates_.add(value);
+          onChanged();
+        } else {
+          updatesBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder addUpdates(
+          int index, org.apache.calcite.avatica.proto.Requests.UpdateBatch value) {
+        if (updatesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureUpdatesIsMutable();
+          updates_.add(index, value);
+          onChanged();
+        } else {
+          updatesBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder addUpdates(
+          org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder builderForValue) {
+        if (updatesBuilder_ == null) {
+          ensureUpdatesIsMutable();
+          updates_.add(builderForValue.build());
+          onChanged();
+        } else {
+          updatesBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder addUpdates(
+          int index, org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder builderForValue) {
+        if (updatesBuilder_ == null) {
+          ensureUpdatesIsMutable();
+          updates_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          updatesBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder addAllUpdates(
+          java.lang.Iterable<? extends org.apache.calcite.avatica.proto.Requests.UpdateBatch> values) {
+        if (updatesBuilder_ == null) {
+          ensureUpdatesIsMutable();
+          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, updates_);
+          onChanged();
+        } else {
+          updatesBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder clearUpdates() {
+        if (updatesBuilder_ == null) {
+          updates_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+          onChanged();
+        } else {
+          updatesBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public Builder removeUpdates(int index) {
+        if (updatesBuilder_ == null) {
+          ensureUpdatesIsMutable();
+          updates_.remove(index);
+          onChanged();
+        } else {
+          updatesBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder getUpdatesBuilder(
+          int index) {
+        return getUpdatesFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder getUpdatesOrBuilder(
+          int index) {
+        if (updatesBuilder_ == null) {
+          return updates_.get(index);  } else {
+          return updatesBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public java.util.List<? extends org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder> 
+           getUpdatesOrBuilderList() {
+        if (updatesBuilder_ != null) {
+          return updatesBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(updates_);
+        }
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder addUpdatesBuilder() {
+        return getUpdatesFieldBuilder().addBuilder(
+            org.apache.calcite.avatica.proto.Requests.UpdateBatch.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder addUpdatesBuilder(
+          int index) {
+        return getUpdatesFieldBuilder().addBuilder(
+            index, org.apache.calcite.avatica.proto.Requests.UpdateBatch.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .UpdateBatch updates = 3;</code>
+       *
+       * <pre>
+       * A batch of updates is a list&lt;list&lt;typevalue&gt;&gt;
+       * </pre>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder> 
+           getUpdatesBuilderList() {
+        return getUpdatesFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.calcite.avatica.proto.Requests.UpdateBatch, org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder, org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder> 
+          getUpdatesFieldBuilder() {
+        if (updatesBuilder_ == null) {
+          updatesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.calcite.avatica.proto.Requests.UpdateBatch, org.apache.calcite.avatica.proto.Requests.UpdateBatch.Builder, org.apache.calcite.avatica.proto.Requests.UpdateBatchOrBuilder>(
+                  updates_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          updates_ = null;
+        }
+        return updatesBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:ExecuteBatchRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:ExecuteBatchRequest)
+    private static final org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest();
+    }
+
+    public static org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<ExecuteBatchRequest>
+        PARSER = new com.google.protobuf.AbstractParser<ExecuteBatchRequest>() {
+      public ExecuteBatchRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        try {
+          return new ExecuteBatchRequest(input, extensionRegistry);
+        } catch (RuntimeException e) {
+          if (e.getCause() instanceof
+              com.google.protobuf.InvalidProtocolBufferException) {
+            throw (com.google.protobuf.InvalidProtocolBufferException)
+                e.getCause();
+          }
+          throw e;
+        }
+      }
+    };
+
+    public static com.google.protobuf.Parser<ExecuteBatchRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ExecuteBatchRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_CatalogsRequest_descriptor;
   private static
@@ -12135,6 +14510,21 @@ package org.apache.calcite.avatica.proto;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_RollbackRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_PrepareAndExecuteBatchRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_PrepareAndExecuteBatchRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_UpdateBatch_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_UpdateBatch_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ExecuteBatchRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ExecuteBatchRequest_fieldAccessorTable;
 
   public stat

<TRUNCATED>

[2/4] calcite git commit: [CALCITE-1128] Implement JDBC batch update methods in remote driver

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Responses.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Responses.java b/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Responses.java
index cd3d0cf..a26e096 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Responses.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/proto/Responses.java
@@ -11607,6 +11607,957 @@ package org.apache.calcite.avatica.proto;
 
   }
 
+  public interface ExecuteBatchResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:ExecuteBatchResponse)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    java.lang.String getConnectionId();
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getConnectionIdBytes();
+
+    /**
+     * <code>optional uint32 statement_id = 2;</code>
+     */
+    int getStatementId();
+
+    /**
+     * <code>repeated uint32 update_counts = 3;</code>
+     */
+    java.util.List<java.lang.Integer> getUpdateCountsList();
+    /**
+     * <code>repeated uint32 update_counts = 3;</code>
+     */
+    int getUpdateCountsCount();
+    /**
+     * <code>repeated uint32 update_counts = 3;</code>
+     */
+    int getUpdateCounts(int index);
+
+    /**
+     * <code>optional bool missing_statement = 4;</code>
+     *
+     * <pre>
+     * Did the request fail because of no-cached statement
+     * </pre>
+     */
+    boolean getMissingStatement();
+
+    /**
+     * <code>optional .RpcMetadata metadata = 5;</code>
+     */
+    boolean hasMetadata();
+    /**
+     * <code>optional .RpcMetadata metadata = 5;</code>
+     */
+    org.apache.calcite.avatica.proto.Responses.RpcMetadata getMetadata();
+    /**
+     * <code>optional .RpcMetadata metadata = 5;</code>
+     */
+    org.apache.calcite.avatica.proto.Responses.RpcMetadataOrBuilder getMetadataOrBuilder();
+  }
+  /**
+   * Protobuf type {@code ExecuteBatchResponse}
+   *
+   * <pre>
+   * Response to a batch update request
+   * </pre>
+   */
+  public  static final class ExecuteBatchResponse extends
+      com.google.protobuf.GeneratedMessage implements
+      // @@protoc_insertion_point(message_implements:ExecuteBatchResponse)
+      ExecuteBatchResponseOrBuilder {
+    // Use ExecuteBatchResponse.newBuilder() to construct.
+    private ExecuteBatchResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+    }
+    private ExecuteBatchResponse() {
+      connectionId_ = "";
+      statementId_ = 0;
+      updateCounts_ = java.util.Collections.emptyList();
+      missingStatement_ = false;
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private ExecuteBatchResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry) {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              String s = input.readStringRequireUtf8();
+
+              connectionId_ = s;
+              break;
+            }
+            case 16: {
+
+              statementId_ = input.readUInt32();
+              break;
+            }
+            case 24: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                updateCounts_ = new java.util.ArrayList<java.lang.Integer>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              updateCounts_.add(input.readUInt32());
+              break;
+            }
+            case 26: {
+              int length = input.readRawVarint32();
+              int limit = input.pushLimit(length);
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004) && input.getBytesUntilLimit() > 0) {
+                updateCounts_ = new java.util.ArrayList<java.lang.Integer>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              while (input.getBytesUntilLimit() > 0) {
+                updateCounts_.add(input.readUInt32());
+              }
+              input.popLimit(limit);
+              break;
+            }
+            case 32: {
+
+              missingStatement_ = input.readBool();
+              break;
+            }
+            case 42: {
+              org.apache.calcite.avatica.proto.Responses.RpcMetadata.Builder subBuilder = null;
+              if (metadata_ != null) {
+                subBuilder = metadata_.toBuilder();
+              }
+              metadata_ = input.readMessage(org.apache.calcite.avatica.proto.Responses.RpcMetadata.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(metadata_);
+                metadata_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw new RuntimeException(e.setUnfinishedMessage(this));
+      } catch (java.io.IOException e) {
+        throw new RuntimeException(
+            new com.google.protobuf.InvalidProtocolBufferException(
+                e.getMessage()).setUnfinishedMessage(this));
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          updateCounts_ = java.util.Collections.unmodifiableList(updateCounts_);
+        }
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.calcite.avatica.proto.Responses.internal_static_ExecuteBatchResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.calcite.avatica.proto.Responses.internal_static_ExecuteBatchResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse.class, org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int CONNECTION_ID_FIELD_NUMBER = 1;
+    private volatile java.lang.Object connectionId_;
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    public java.lang.String getConnectionId() {
+      java.lang.Object ref = connectionId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        connectionId_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>optional string connection_id = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getConnectionIdBytes() {
+      java.lang.Object ref = connectionId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        connectionId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int STATEMENT_ID_FIELD_NUMBER = 2;
+    private int statementId_;
+    /**
+     * <code>optional uint32 statement_id = 2;</code>
+     */
+    public int getStatementId() {
+      return statementId_;
+    }
+
+    public static final int UPDATE_COUNTS_FIELD_NUMBER = 3;
+    private java.util.List<java.lang.Integer> updateCounts_;
+    /**
+     * <code>repeated uint32 update_counts = 3;</code>
+     */
+    public java.util.List<java.lang.Integer>
+        getUpdateCountsList() {
+      return updateCounts_;
+    }
+    /**
+     * <code>repeated uint32 update_counts = 3;</code>
+     */
+    public int getUpdateCountsCount() {
+      return updateCounts_.size();
+    }
+    /**
+     * <code>repeated uint32 update_counts = 3;</code>
+     */
+    public int getUpdateCounts(int index) {
+      return updateCounts_.get(index);
+    }
+    private int updateCountsMemoizedSerializedSize = -1;
+
+    public static final int MISSING_STATEMENT_FIELD_NUMBER = 4;
+    private boolean missingStatement_;
+    /**
+     * <code>optional bool missing_statement = 4;</code>
+     *
+     * <pre>
+     * Did the request fail because of no-cached statement
+     * </pre>
+     */
+    public boolean getMissingStatement() {
+      return missingStatement_;
+    }
+
+    public static final int METADATA_FIELD_NUMBER = 5;
+    private org.apache.calcite.avatica.proto.Responses.RpcMetadata metadata_;
+    /**
+     * <code>optional .RpcMetadata metadata = 5;</code>
+     */
+    public boolean hasMetadata() {
+      return metadata_ != null;
+    }
+    /**
+     * <code>optional .RpcMetadata metadata = 5;</code>
+     */
+    public org.apache.calcite.avatica.proto.Responses.RpcMetadata getMetadata() {
+      return metadata_ == null ? org.apache.calcite.avatica.proto.Responses.RpcMetadata.getDefaultInstance() : metadata_;
+    }
+    /**
+     * <code>optional .RpcMetadata metadata = 5;</code>
+     */
+    public org.apache.calcite.avatica.proto.Responses.RpcMetadataOrBuilder getMetadataOrBuilder() {
+      return getMetadata();
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (!getConnectionIdBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessage.writeString(output, 1, connectionId_);
+      }
+      if (statementId_ != 0) {
+        output.writeUInt32(2, statementId_);
+      }
+      if (getUpdateCountsList().size() > 0) {
+        output.writeRawVarint32(26);
+        output.writeRawVarint32(updateCountsMemoizedSerializedSize);
+      }
+      for (int i = 0; i < updateCounts_.size(); i++) {
+        output.writeUInt32NoTag(updateCounts_.get(i));
+      }
+      if (missingStatement_ != false) {
+        output.writeBool(4, missingStatement_);
+      }
+      if (metadata_ != null) {
+        output.writeMessage(5, getMetadata());
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (!getConnectionIdBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessage.computeStringSize(1, connectionId_);
+      }
+      if (statementId_ != 0) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(2, statementId_);
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < updateCounts_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeUInt32SizeNoTag(updateCounts_.get(i));
+        }
+        size += dataSize;
+        if (!getUpdateCountsList().isEmpty()) {
+          size += 1;
+          size += com.google.protobuf.CodedOutputStream
+              .computeInt32SizeNoTag(dataSize);
+        }
+        updateCountsMemoizedSerializedSize = dataSize;
+      }
+      if (missingStatement_ != false) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(4, missingStatement_);
+      }
+      if (metadata_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, getMetadata());
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ExecuteBatchResponse}
+     *
+     * <pre>
+     * Response to a batch update request
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:ExecuteBatchResponse)
+        org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.calcite.avatica.proto.Responses.internal_static_ExecuteBatchResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.calcite.avatica.proto.Responses.internal_static_ExecuteBatchResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse.class, org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse.Builder.class);
+      }
+
+      // Construct using org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        connectionId_ = "";
+
+        statementId_ = 0;
+
+        updateCounts_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000004);
+        missingStatement_ = false;
+
+        if (metadataBuilder_ == null) {
+          metadata_ = null;
+        } else {
+          metadata_ = null;
+          metadataBuilder_ = null;
+        }
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.calcite.avatica.proto.Responses.internal_static_ExecuteBatchResponse_descriptor;
+      }
+
+      public org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse getDefaultInstanceForType() {
+        return org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse.getDefaultInstance();
+      }
+
+      public org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse build() {
+        org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse buildPartial() {
+        org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse result = new org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        result.connectionId_ = connectionId_;
+        result.statementId_ = statementId_;
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          updateCounts_ = java.util.Collections.unmodifiableList(updateCounts_);
+          bitField0_ = (bitField0_ & ~0x00000004);
+        }
+        result.updateCounts_ = updateCounts_;
+        result.missingStatement_ = missingStatement_;
+        if (metadataBuilder_ == null) {
+          result.metadata_ = metadata_;
+        } else {
+          result.metadata_ = metadataBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse) {
+          return mergeFrom((org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse other) {
+        if (other == org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse.getDefaultInstance()) return this;
+        if (!other.getConnectionId().isEmpty()) {
+          connectionId_ = other.connectionId_;
+          onChanged();
+        }
+        if (other.getStatementId() != 0) {
+          setStatementId(other.getStatementId());
+        }
+        if (!other.updateCounts_.isEmpty()) {
+          if (updateCounts_.isEmpty()) {
+            updateCounts_ = other.updateCounts_;
+            bitField0_ = (bitField0_ & ~0x00000004);
+          } else {
+            ensureUpdateCountsIsMutable();
+            updateCounts_.addAll(other.updateCounts_);
+          }
+          onChanged();
+        }
+        if (other.getMissingStatement() != false) {
+          setMissingStatement(other.getMissingStatement());
+        }
+        if (other.hasMetadata()) {
+          mergeMetadata(other.getMetadata());
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.lang.Object connectionId_ = "";
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public java.lang.String getConnectionId() {
+        java.lang.Object ref = connectionId_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          connectionId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getConnectionIdBytes() {
+        java.lang.Object ref = connectionId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          connectionId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder setConnectionId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        connectionId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder clearConnectionId() {
+        
+        connectionId_ = getDefaultInstance().getConnectionId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string connection_id = 1;</code>
+       */
+      public Builder setConnectionIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        connectionId_ = value;
+        onChanged();
+        return this;
+      }
+
+      private int statementId_ ;
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public int getStatementId() {
+        return statementId_;
+      }
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public Builder setStatementId(int value) {
+        
+        statementId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 statement_id = 2;</code>
+       */
+      public Builder clearStatementId() {
+        
+        statementId_ = 0;
+        onChanged();
+        return this;
+      }
+
+      private java.util.List<java.lang.Integer> updateCounts_ = java.util.Collections.emptyList();
+      private void ensureUpdateCountsIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          updateCounts_ = new java.util.ArrayList<java.lang.Integer>(updateCounts_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+      /**
+       * <code>repeated uint32 update_counts = 3;</code>
+       */
+      public java.util.List<java.lang.Integer>
+          getUpdateCountsList() {
+        return java.util.Collections.unmodifiableList(updateCounts_);
+      }
+      /**
+       * <code>repeated uint32 update_counts = 3;</code>
+       */
+      public int getUpdateCountsCount() {
+        return updateCounts_.size();
+      }
+      /**
+       * <code>repeated uint32 update_counts = 3;</code>
+       */
+      public int getUpdateCounts(int index) {
+        return updateCounts_.get(index);
+      }
+      /**
+       * <code>repeated uint32 update_counts = 3;</code>
+       */
+      public Builder setUpdateCounts(
+          int index, int value) {
+        ensureUpdateCountsIsMutable();
+        updateCounts_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated uint32 update_counts = 3;</code>
+       */
+      public Builder addUpdateCounts(int value) {
+        ensureUpdateCountsIsMutable();
+        updateCounts_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated uint32 update_counts = 3;</code>
+       */
+      public Builder addAllUpdateCounts(
+          java.lang.Iterable<? extends java.lang.Integer> values) {
+        ensureUpdateCountsIsMutable();
+        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            values, updateCounts_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated uint32 update_counts = 3;</code>
+       */
+      public Builder clearUpdateCounts() {
+        updateCounts_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000004);
+        onChanged();
+        return this;
+      }
+
+      private boolean missingStatement_ ;
+      /**
+       * <code>optional bool missing_statement = 4;</code>
+       *
+       * <pre>
+       * Did the request fail because of no-cached statement
+       * </pre>
+       */
+      public boolean getMissingStatement() {
+        return missingStatement_;
+      }
+      /**
+       * <code>optional bool missing_statement = 4;</code>
+       *
+       * <pre>
+       * Did the request fail because of no-cached statement
+       * </pre>
+       */
+      public Builder setMissingStatement(boolean value) {
+        
+        missingStatement_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool missing_statement = 4;</code>
+       *
+       * <pre>
+       * Did the request fail because of no-cached statement
+       * </pre>
+       */
+      public Builder clearMissingStatement() {
+        
+        missingStatement_ = false;
+        onChanged();
+        return this;
+      }
+
+      private org.apache.calcite.avatica.proto.Responses.RpcMetadata metadata_ = null;
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.calcite.avatica.proto.Responses.RpcMetadata, org.apache.calcite.avatica.proto.Responses.RpcMetadata.Builder, org.apache.calcite.avatica.proto.Responses.RpcMetadataOrBuilder> metadataBuilder_;
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public boolean hasMetadata() {
+        return metadataBuilder_ != null || metadata_ != null;
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public org.apache.calcite.avatica.proto.Responses.RpcMetadata getMetadata() {
+        if (metadataBuilder_ == null) {
+          return metadata_ == null ? org.apache.calcite.avatica.proto.Responses.RpcMetadata.getDefaultInstance() : metadata_;
+        } else {
+          return metadataBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public Builder setMetadata(org.apache.calcite.avatica.proto.Responses.RpcMetadata value) {
+        if (metadataBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          metadata_ = value;
+          onChanged();
+        } else {
+          metadataBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public Builder setMetadata(
+          org.apache.calcite.avatica.proto.Responses.RpcMetadata.Builder builderForValue) {
+        if (metadataBuilder_ == null) {
+          metadata_ = builderForValue.build();
+          onChanged();
+        } else {
+          metadataBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public Builder mergeMetadata(org.apache.calcite.avatica.proto.Responses.RpcMetadata value) {
+        if (metadataBuilder_ == null) {
+          if (metadata_ != null) {
+            metadata_ =
+              org.apache.calcite.avatica.proto.Responses.RpcMetadata.newBuilder(metadata_).mergeFrom(value).buildPartial();
+          } else {
+            metadata_ = value;
+          }
+          onChanged();
+        } else {
+          metadataBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public Builder clearMetadata() {
+        if (metadataBuilder_ == null) {
+          metadata_ = null;
+          onChanged();
+        } else {
+          metadata_ = null;
+          metadataBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public org.apache.calcite.avatica.proto.Responses.RpcMetadata.Builder getMetadataBuilder() {
+        
+        onChanged();
+        return getMetadataFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      public org.apache.calcite.avatica.proto.Responses.RpcMetadataOrBuilder getMetadataOrBuilder() {
+        if (metadataBuilder_ != null) {
+          return metadataBuilder_.getMessageOrBuilder();
+        } else {
+          return metadata_ == null ?
+              org.apache.calcite.avatica.proto.Responses.RpcMetadata.getDefaultInstance() : metadata_;
+        }
+      }
+      /**
+       * <code>optional .RpcMetadata metadata = 5;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.calcite.avatica.proto.Responses.RpcMetadata, org.apache.calcite.avatica.proto.Responses.RpcMetadata.Builder, org.apache.calcite.avatica.proto.Responses.RpcMetadataOrBuilder> 
+          getMetadataFieldBuilder() {
+        if (metadataBuilder_ == null) {
+          metadataBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.calcite.avatica.proto.Responses.RpcMetadata, org.apache.calcite.avatica.proto.Responses.RpcMetadata.Builder, org.apache.calcite.avatica.proto.Responses.RpcMetadataOrBuilder>(
+                  getMetadata(),
+                  getParentForChildren(),
+                  isClean());
+          metadata_ = null;
+        }
+        return metadataBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:ExecuteBatchResponse)
+    }
+
+    // @@protoc_insertion_point(class_scope:ExecuteBatchResponse)
+    private static final org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse();
+    }
+
+    public static org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<ExecuteBatchResponse>
+        PARSER = new com.google.protobuf.AbstractParser<ExecuteBatchResponse>() {
+      public ExecuteBatchResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        try {
+          return new ExecuteBatchResponse(input, extensionRegistry);
+        } catch (RuntimeException e) {
+          if (e.getCause() instanceof
+              com.google.protobuf.InvalidProtocolBufferException) {
+            throw (com.google.protobuf.InvalidProtocolBufferException)
+                e.getCause();
+          }
+          throw e;
+        }
+      }
+    };
+
+    public static com.google.protobuf.Parser<ExecuteBatchResponse> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ExecuteBatchResponse> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_ResultSetResponse_descriptor;
   private static
@@ -11687,6 +12638,11 @@ package org.apache.calcite.avatica.proto;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_RollbackResponse_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ExecuteBatchResponse_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ExecuteBatchResponse_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -11734,8 +12690,12 @@ package org.apache.calcite.avatica.proto;
       "\014more_results\030\002 \001(\010\022\036\n\010metadata\030\003 \001(\0132\014." +
       "RpcMetadata\"%\n\013RpcMetadata\022\026\n\016server_add" +
       "ress\030\001 \001(\t\"\020\n\016CommitResponse\"\022\n\020Rollback" +
-      "ResponseB\"\n org.apache.calcite.avatica.p" +
-      "rotob\006proto3"
+      "Response\"\225\001\n\024ExecuteBatchResponse\022\025\n\rcon" +
+      "nection_id\030\001 \001(\t\022\024\n\014statement_id\030\002 \001(\r\022\025",
+      "\n\rupdate_counts\030\003 \003(\r\022\031\n\021missing_stateme" +
+      "nt\030\004 \001(\010\022\036\n\010metadata\030\005 \001(\0132\014.RpcMetadata" +
+      "B\"\n org.apache.calcite.avatica.protob\006pr" +
+      "oto3"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -11846,6 +12806,12 @@ package org.apache.calcite.avatica.proto;
       com.google.protobuf.GeneratedMessage.FieldAccessorTable(
         internal_static_RollbackResponse_descriptor,
         new java.lang.String[] { });
+    internal_static_ExecuteBatchResponse_descriptor =
+      getDescriptor().getMessageTypes().get(16);
+    internal_static_ExecuteBatchResponse_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+        internal_static_ExecuteBatchResponse_descriptor,
+        new java.lang.String[] { "ConnectionId", "StatementId", "UpdateCounts", "MissingStatement", "Metadata", });
     org.apache.calcite.avatica.proto.Common.getDescriptor();
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/JsonService.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
index 668b3be..19c95e7 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
@@ -222,6 +222,22 @@ public abstract class JsonService extends AbstractService {
       throw handle(e);
     }
   }
+
+  public ExecuteBatchResponse apply(PrepareAndExecuteBatchRequest request) {
+    try {
+      return decode(apply(encode(request)), ExecuteBatchResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
+
+  public ExecuteBatchResponse apply(ExecuteBatchRequest request) {
+    try {
+      return decode(apply(encode(request)), ExecuteBatchResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
 }
 
 // End JsonService.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
index c070ec0..a15d55f 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
@@ -18,6 +18,7 @@ package org.apache.calcite.avatica.remote;
 
 import org.apache.calcite.avatica.Meta;
 
+import org.apache.calcite.avatica.Meta.ExecuteBatchResult;
 import org.apache.calcite.avatica.MetaImpl;
 import org.apache.calcite.avatica.MissingResultsException;
 import org.apache.calcite.avatica.NoSuchStatementException;
@@ -353,6 +354,37 @@ public class LocalService implements Service {
     // If rollback() errors, let the ErrorResponse be sent back via an uncaught Exception.
     return new RollbackResponse();
   }
+
+  public ExecuteBatchResponse apply(PrepareAndExecuteBatchRequest request) {
+    final Meta.StatementHandle h = new Meta.StatementHandle(request.connectionId,
+        request.statementId, null);
+    try {
+      ExecuteBatchResult result = meta.prepareAndExecuteBatch(h, request.sqlCommands);
+      return new ExecuteBatchResponse(request.connectionId, request.statementId,
+          result.updateCounts, false, serverLevelRpcMetadata);
+    } catch (NoSuchStatementException e) {
+      return new ExecuteBatchResponse(request.connectionId, request.statementId, null, true,
+          serverLevelRpcMetadata);
+    }
+  }
+
+  public ExecuteBatchResponse apply(ExecuteBatchRequest request) {
+    final Meta.StatementHandle h = new Meta.StatementHandle(request.connectionId,
+        request.statementId, null);
+    try {
+      ExecuteBatchResult result;
+      if (request.hasProtoUpdateBatches() && meta instanceof ProtobufMeta) {
+        result = ((ProtobufMeta) meta).executeBatchProtobuf(h, request.getProtoUpdateBatches());
+      } else {
+        result = meta.executeBatch(h, request.parameterValues);
+      }
+      return new ExecuteBatchResponse(request.connectionId, request.statementId,
+          result.updateCounts, false, serverLevelRpcMetadata);
+    } catch (NoSuchStatementException e) {
+      return new ExecuteBatchResponse(request.connectionId, request.statementId, null, true,
+          serverLevelRpcMetadata);
+    }
+  }
 }
 
 // End LocalService.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufMeta.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufMeta.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufMeta.java
new file mode 100644
index 0000000..375ae80
--- /dev/null
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufMeta.java
@@ -0,0 +1,45 @@
+/*
+ * 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.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.NoSuchStatementException;
+import org.apache.calcite.avatica.proto.Requests;
+
+import java.util.List;
+
+/**
+ * An extension of {@link Meta} which allows for native processing of calls with the Protobuf
+ * API objects instead of the POJOS (to avoid object translation). In the write-path, performing
+ * this conversion tends to represent a signficant portion of execution time. The introduction
+ * of this interface is to serve the purose of gradual migration to Meta implementations that
+ * can naturally function over Protobuf objects instead of the POJOs.
+ */
+public interface ProtobufMeta extends Meta {
+
+  /**
+   * Executes a batch of commands on a prepared statement.
+   *
+   * @param h Statement handle
+   * @param parameterValues A collection of list of typed values, one list per batch
+   * @return An array of update counts containing one element for each command in the batch.
+   */
+  ExecuteBatchResult executeBatchProtobuf(StatementHandle h, List<Requests.UpdateBatch>
+      parameterValues) throws NoSuchStatementException;
+}
+
+// End ProtobufMeta.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
index 56ba125..d694440 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
@@ -109,6 +109,14 @@ public abstract class ProtobufService extends AbstractService {
     return (RollbackResponse) _apply(request);
   }
 
+  @Override public ExecuteBatchResponse apply(PrepareAndExecuteBatchRequest request) {
+    return (ExecuteBatchResponse) _apply(request);
+  }
+
+  @Override public ExecuteBatchResponse apply(ExecuteBatchRequest request) {
+    return (ExecuteBatchResponse) _apply(request);
+  }
+
   /**
    * Checks if the provided {@link Message} is an instance of the Class given by
    * <code>expectedType</code>. Throws an IllegalArgumentException if the message is not of the

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
index 80d2b22..b9c57c5 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
@@ -25,9 +25,11 @@ import org.apache.calcite.avatica.proto.Requests.CommitRequest;
 import org.apache.calcite.avatica.proto.Requests.ConnectionSyncRequest;
 import org.apache.calcite.avatica.proto.Requests.CreateStatementRequest;
 import org.apache.calcite.avatica.proto.Requests.DatabasePropertyRequest;
+import org.apache.calcite.avatica.proto.Requests.ExecuteBatchRequest;
 import org.apache.calcite.avatica.proto.Requests.ExecuteRequest;
 import org.apache.calcite.avatica.proto.Requests.FetchRequest;
 import org.apache.calcite.avatica.proto.Requests.OpenConnectionRequest;
+import org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteBatchRequest;
 import org.apache.calcite.avatica.proto.Requests.PrepareAndExecuteRequest;
 import org.apache.calcite.avatica.proto.Requests.PrepareRequest;
 import org.apache.calcite.avatica.proto.Requests.RollbackRequest;
@@ -43,6 +45,7 @@ import org.apache.calcite.avatica.proto.Responses.ConnectionSyncResponse;
 import org.apache.calcite.avatica.proto.Responses.CreateStatementResponse;
 import org.apache.calcite.avatica.proto.Responses.DatabasePropertyResponse;
 import org.apache.calcite.avatica.proto.Responses.ErrorResponse;
+import org.apache.calcite.avatica.proto.Responses.ExecuteBatchResponse;
 import org.apache.calcite.avatica.proto.Responses.ExecuteResponse;
 import org.apache.calcite.avatica.proto.Responses.FetchResponse;
 import org.apache.calcite.avatica.proto.Responses.OpenConnectionResponse;
@@ -126,6 +129,12 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
         new RequestTranslator(CommitRequest.parser(), new Service.CommitRequest()));
     reqParsers.put(RollbackRequest.class.getName(),
         new RequestTranslator(RollbackRequest.parser(), new Service.RollbackRequest()));
+    reqParsers.put(PrepareAndExecuteBatchRequest.class.getName(),
+        new RequestTranslator(PrepareAndExecuteBatchRequest.parser(),
+            new Service.PrepareAndExecuteBatchRequest()));
+    reqParsers.put(ExecuteBatchRequest.class.getName(),
+        new RequestTranslator(ExecuteBatchRequest.parser(),
+            new Service.ExecuteBatchRequest()));
 
     REQUEST_PARSERS = Collections.unmodifiableMap(reqParsers);
 
@@ -166,6 +175,8 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
         new ResponseTranslator(CommitResponse.parser(), new Service.CommitResponse()));
     respParsers.put(RollbackResponse.class.getName(),
         new ResponseTranslator(RollbackResponse.parser(), new Service.RollbackResponse()));
+    respParsers.put(ExecuteBatchResponse.class.getName(),
+        new ResponseTranslator(ExecuteBatchResponse.parser(), new Service.ExecuteBatchResponse()));
 
     RESPONSE_PARSERS = Collections.unmodifiableMap(respParsers);
 
@@ -197,6 +208,9 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
     messageClasses.add(TableTypesRequest.class);
     messageClasses.add(TablesRequest.class);
     messageClasses.add(TypeInfoRequest.class);
+    messageClasses.add(PrepareAndExecuteBatchRequest.class);
+    messageClasses.add(ExecuteBatchRequest.class);
+
     messageClasses.add(CloseConnectionResponse.class);
     messageClasses.add(CloseStatementResponse.class);
     messageClasses.add(CommitResponse.class);
@@ -212,6 +226,7 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
     messageClasses.add(RollbackResponse.class);
     messageClasses.add(RpcMetadata.class);
     messageClasses.add(SyncResultsResponse.class);
+    messageClasses.add(ExecuteBatchResponse.class);
 
     return messageClasses;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
index 463985a..152e0ca 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
@@ -390,6 +390,29 @@ class RemoteMeta extends MetaImpl {
       }
     });
   }
+
+  @Override public ExecuteBatchResult prepareAndExecuteBatch(final StatementHandle h,
+      final List<String> sqlCommands) throws NoSuchStatementException {
+    return connection.invokeWithRetries(new CallableWithoutException<ExecuteBatchResult>() {
+      @Override public ExecuteBatchResult call() {
+        Service.ExecuteBatchResponse response =
+            service.apply(
+                new Service.PrepareAndExecuteBatchRequest(h.connectionId, h.id, sqlCommands));
+        return new ExecuteBatchResult(response.updateCounts);
+      }
+    });
+  }
+
+  @Override public ExecuteBatchResult executeBatch(final StatementHandle h,
+      final List<List<TypedValue>> parameterValues) throws NoSuchStatementException {
+    return connection.invokeWithRetries(new CallableWithoutException<ExecuteBatchResult>() {
+      @Override public ExecuteBatchResult call() {
+        Service.ExecuteBatchResponse response =
+            service.apply(new Service.ExecuteBatchRequest(h.connectionId, h.id, parameterValues));
+        return new ExecuteBatchResult(response.updateCounts);
+      }
+    });
+  }
 }
 
 // End RemoteMeta.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/Service.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
index 5790848..078e63e 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -28,6 +28,7 @@ import org.apache.calcite.avatica.proto.Requests;
 import org.apache.calcite.avatica.proto.Responses;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
@@ -40,6 +41,7 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -71,6 +73,8 @@ public interface Service {
   DatabasePropertyResponse apply(DatabasePropertyRequest request);
   CommitResponse apply(CommitRequest request);
   RollbackResponse apply(RollbackRequest request);
+  ExecuteBatchResponse apply(PrepareAndExecuteBatchRequest request);
+  ExecuteBatchResponse apply(ExecuteBatchRequest request);
 
   /**
    * Sets server-level metadata for RPCs. This includes information that is static across all RPCs.
@@ -134,7 +138,10 @@ public interface Service {
       @JsonSubTypes.Type(value = DatabasePropertyRequest.class, name = "databaseProperties"),
       @JsonSubTypes.Type(value = SyncResultsRequest.class, name = "syncResults"),
       @JsonSubTypes.Type(value = CommitRequest.class, name = "commit"),
-      @JsonSubTypes.Type(value = RollbackRequest.class, name = "rollback") })
+      @JsonSubTypes.Type(value = RollbackRequest.class, name = "rollback"),
+      @JsonSubTypes.Type(value = PrepareAndExecuteBatchRequest.class,
+          name = "prepareAndExecuteBatch"),
+      @JsonSubTypes.Type(value = ExecuteBatchRequest.class, name = "executeBatch") })
   abstract class Request extends Base {
     abstract Response accept(Service service);
     abstract Request deserialize(Message genericMsg);
@@ -164,7 +171,8 @@ public interface Service {
       @JsonSubTypes.Type(value = SyncResultsResponse.class, name = "syncResults"),
       @JsonSubTypes.Type(value = RpcMetadataResponse.class, name = "rpcMetadata"),
       @JsonSubTypes.Type(value = CommitResponse.class, name = "commit"),
-      @JsonSubTypes.Type(value = RollbackResponse.class, name = "rollback") })
+      @JsonSubTypes.Type(value = RollbackResponse.class, name = "rollback"),
+      @JsonSubTypes.Type(value = ExecuteBatchResponse.class, name = "executeBatch") })
   abstract class Response extends Base {
     abstract Response deserialize(Message genericMsg);
     abstract Message serialize();
@@ -2766,6 +2774,270 @@ public interface Service {
     }
   }
 
+  /**
+   * Request to prepare a statement and execute a series of batch commands in one call.
+   */
+  class PrepareAndExecuteBatchRequest extends Request {
+    public final String connectionId;
+    public final List<String> sqlCommands;
+    public final int statementId;
+
+    PrepareAndExecuteBatchRequest() {
+      connectionId = null;
+      statementId = 0;
+      sqlCommands = null;
+    }
+
+    @JsonCreator
+    public PrepareAndExecuteBatchRequest(@JsonProperty("connectionId") String connectionId,
+        @JsonProperty("statementId") int statementId, @JsonProperty("sqlCommands") List<String>
+        sqlCommands) {
+      this.connectionId = connectionId;
+      this.sqlCommands = sqlCommands;
+      this.statementId = statementId;
+    }
+
+    @Override public ExecuteBatchResponse accept(Service service) {
+      return service.apply(this);
+    }
+
+    @Override public Requests.PrepareAndExecuteBatchRequest serialize() {
+      Requests.PrepareAndExecuteBatchRequest.Builder builder =
+          Requests.PrepareAndExecuteBatchRequest.newBuilder();
+
+      if (null != connectionId) {
+        builder.setConnectionId(connectionId);
+      }
+
+      if (null != sqlCommands) {
+        builder.addAllSqlCommands(sqlCommands);
+      }
+
+      return builder.setStatementId(statementId).build();
+    }
+
+    @Override public PrepareAndExecuteBatchRequest deserialize(Message genericMsg) {
+      final Requests.PrepareAndExecuteBatchRequest msg =
+          ProtobufService.castProtobufMessage(genericMsg,
+              Requests.PrepareAndExecuteBatchRequest.class);
+
+      List<String> sqlCommands = new ArrayList<>(msg.getSqlCommandsList());
+
+      return new PrepareAndExecuteBatchRequest(msg.getConnectionId(), msg.getStatementId(),
+          sqlCommands);
+    }
+
+    @Override public int hashCode() {
+      int result = 1;
+      result = p(result, connectionId);
+      result = p(result, statementId);
+      result = p(result, sqlCommands);
+      return result;
+    }
+
+    @Override public boolean equals(Object o) {
+      return this == o
+          || o instanceof PrepareAndExecuteBatchRequest
+          && Objects.equals(connectionId, ((PrepareAndExecuteBatchRequest) o).connectionId)
+          && statementId == ((PrepareAndExecuteBatchRequest) o).statementId
+          && Objects.equals(sqlCommands, ((PrepareAndExecuteBatchRequest) o).sqlCommands);
+
+    }
+  }
+
+  /**
+   * Request object to execute a batch of commands.
+   */
+  class ExecuteBatchRequest extends Request {
+    private static final FieldDescriptor UPDATE_BATCH_FIELD_DESCRIPTOR = Requests
+        .ExecuteBatchRequest.getDescriptor()
+        .findFieldByNumber(Requests.ExecuteBatchRequest.UPDATES_FIELD_NUMBER);
+
+    public final String connectionId;
+    public final int statementId;
+    // Each update in a batch has a list of TypedValue's
+    public final List<List<TypedValue>> parameterValues;
+    // Avoid deserializing every parameter list from pb to pojo
+    @JsonIgnore
+    private List<Requests.UpdateBatch> protoParameterValues = null;
+
+    ExecuteBatchRequest() {
+      this.connectionId = null;
+      this.statementId = 0;
+      this.parameterValues = null;
+    }
+
+    @JsonCreator
+    public ExecuteBatchRequest(@JsonProperty("connectionId") String connectionId,
+        @JsonProperty("statementId") int statementId,
+        @JsonProperty("parameterValues") List<List<TypedValue>> parameterValues) {
+      this.connectionId = connectionId;
+      this.statementId = statementId;
+      this.parameterValues = parameterValues;
+    }
+
+    ExecuteBatchRequest(String connectionId, int statementId) {
+      this.connectionId = connectionId;
+      this.statementId = statementId;
+      this.parameterValues = null;
+    }
+
+    /**
+     * Does this instance contain protobuf update batches.
+     * @return True if <code>protoUpdateBatches</code> is non-null.
+     */
+    public boolean hasProtoUpdateBatches() {
+      return null != protoParameterValues;
+    }
+
+    /**
+     * @return The protobuf update batches.
+     */
+    // JsonIgnore on the getter, otherwise Jackson will try to serialize it
+    @JsonIgnore
+    public List<Requests.UpdateBatch> getProtoUpdateBatches() {
+      return protoParameterValues;
+    }
+
+    @Override public ExecuteBatchResponse accept(Service service) {
+      return service.apply(this);
+    }
+
+    @Override ExecuteBatchRequest deserialize(Message genericMsg) {
+      Requests.ExecuteBatchRequest msg = ProtobufService.castProtobufMessage(genericMsg,
+          Requests.ExecuteBatchRequest.class);
+
+      List<Requests.UpdateBatch> updateBatches = msg.getUpdatesList();
+
+      ExecuteBatchRequest pojo =
+          new ExecuteBatchRequest(msg.getConnectionId(), msg.getStatementId());
+      pojo.protoParameterValues = updateBatches;
+      return pojo;
+    }
+
+    @Override Requests.ExecuteBatchRequest serialize() {
+      Requests.ExecuteBatchRequest.Builder builder = Requests.ExecuteBatchRequest.newBuilder();
+
+      if (hasProtoUpdateBatches()) {
+        builder.addAllUpdates(protoParameterValues);
+      } else if (null != parameterValues) {
+        for (List<TypedValue> updateBatch : parameterValues) {
+          Requests.UpdateBatch.Builder batchBuilder = Requests.UpdateBatch.newBuilder();
+          for (TypedValue update : updateBatch) {
+            batchBuilder.addParameterValues(update.toProto());
+          }
+          builder.addUpdates(batchBuilder.build());
+        }
+      }
+
+      return builder.setConnectionId(connectionId).setStatementId(statementId).build();
+    }
+
+    @Override public int hashCode() {
+      int result = 1;
+      result = p(result, connectionId);
+      result = p(result, statementId);
+      result = p(result, parameterValues);
+      return result;
+    }
+
+    @Override public boolean equals(Object o) {
+      return this == o
+          || o instanceof ExecuteBatchRequest
+          && Objects.equals(connectionId, ((ExecuteBatchRequest) o).connectionId)
+          && statementId == ((ExecuteBatchRequest) o).statementId
+          && Objects.equals(protoParameterValues, ((ExecuteBatchRequest) o).protoParameterValues)
+          && Objects.equals(parameterValues, ((ExecuteBatchRequest) o).parameterValues);
+    }
+  }
+
+  /**
+   * Response object for executing a batch of commands.
+   */
+  class ExecuteBatchResponse extends Response {
+    private static final FieldDescriptor RPC_METADATA_DESCRIPTOR = Responses.ExecuteBatchResponse
+        .getDescriptor().findFieldByNumber(Responses.ExecuteBatchResponse.METADATA_FIELD_NUMBER);
+
+    public final String connectionId;
+    public final int statementId;
+    public final int[] updateCounts;
+    public final boolean missingStatement;
+    public final RpcMetadataResponse rpcMetadata;
+
+    ExecuteBatchResponse() {
+      connectionId = null;
+      statementId = 0;
+      updateCounts = null;
+      missingStatement = false;
+      rpcMetadata = null;
+    }
+
+    @JsonCreator
+    public ExecuteBatchResponse(@JsonProperty("connectionId") String connectionId,
+        @JsonProperty("statementId") int statementId,
+        @JsonProperty("updateCounts") int[] updateCounts,
+        @JsonProperty("missingStatement") boolean missingStatement,
+        @JsonProperty("rpcMetadata") RpcMetadataResponse rpcMetadata) {
+      this.connectionId = connectionId;
+      this.statementId = statementId;
+      this.updateCounts = updateCounts;
+      this.missingStatement = missingStatement;
+      this.rpcMetadata = rpcMetadata;
+    }
+
+    @Override public int hashCode() {
+      int result = 1;
+      result = p(result, connectionId);
+      result = p(result, statementId);
+      result = p(result, updateCounts);
+      result = p(result, missingStatement);
+      return result;
+    }
+
+    @Override public boolean equals(Object o) {
+      return this == o
+          || o instanceof ExecuteBatchResponse
+          && Arrays.equals(updateCounts, ((ExecuteBatchResponse) o).updateCounts)
+          && Objects.equals(connectionId, ((ExecuteBatchResponse) o).connectionId)
+          && statementId == ((ExecuteBatchResponse) o).statementId
+          && missingStatement == ((ExecuteBatchResponse) o).missingStatement;
+    }
+
+    @Override ExecuteBatchResponse deserialize(Message genericMsg) {
+      Responses.ExecuteBatchResponse msg = ProtobufService.castProtobufMessage(genericMsg,
+          Responses.ExecuteBatchResponse.class);
+
+      int[] updateCounts = new int[msg.getUpdateCountsCount()];
+      int i = 0;
+      for (Integer updateCount : msg.getUpdateCountsList()) {
+        updateCounts[i++] = updateCount;
+      }
+
+      RpcMetadataResponse metadata = null;
+      if (msg.hasField(RPC_METADATA_DESCRIPTOR)) {
+        metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
+      }
+
+      return new ExecuteBatchResponse(msg.getConnectionId(), msg.getStatementId(), updateCounts,
+          msg.getMissingStatement(), metadata);
+    }
+
+    @Override Responses.ExecuteBatchResponse serialize() {
+      Responses.ExecuteBatchResponse.Builder builder = Responses.ExecuteBatchResponse.newBuilder();
+
+      if (null != updateCounts) {
+        for (int i = 0; i < updateCounts.length; i++) {
+          builder.addUpdateCounts(updateCounts[i]);
+        }
+      }
+
+      if (null != rpcMetadata) {
+        builder.setMetadata(rpcMetadata.serialize());
+      }
+
+      return builder.setConnectionId(connectionId).setStatementId(statementId).build();
+    }
+  }
 }
 
 // End Service.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
index d96293b..1146a47 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
@@ -31,6 +31,7 @@ import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Date;
 import java.util.List;
+import java.util.Objects;
 
 /** Value and type.
  *
@@ -244,6 +245,40 @@ public class TypedValue {
     }
   }
 
+  private static Object protoSerialToLocal(Common.Rep rep, Object value) {
+    switch (rep) {
+    case BYTE:
+      return ((Number) value).byteValue();
+    case SHORT:
+      return ((Number) value).shortValue();
+    case INTEGER:
+    case JAVA_SQL_DATE:
+    case JAVA_SQL_TIME:
+      return ((Number) value).intValue();
+    case LONG:
+    case JAVA_UTIL_DATE:
+    case JAVA_SQL_TIMESTAMP:
+      return ((Number) value).longValue();
+    case FLOAT:
+      return ((Number) value).floatValue();
+    case DOUBLE:
+      return ((Number) value).doubleValue();
+    case NUMBER:
+      return value instanceof BigDecimal ? value
+          : value instanceof BigInteger ? new BigDecimal((BigInteger) value)
+          : value instanceof Double ? new BigDecimal((Double) value)
+          : value instanceof Float ? new BigDecimal((Float) value)
+          : new BigDecimal(((Number) value).longValue());
+    case BYTE_STRING:
+      return (byte[]) value;
+    case STRING:
+      return (String) value;
+    default:
+      throw new IllegalArgumentException("cannot convert " + value + " ("
+          + value.getClass() + ") to " + rep);
+    }
+  }
+
   /** Converts the value into the JDBC representation.
    *
    * <p>For example, a byte string is represented as a {@link ByteString};
@@ -276,6 +311,22 @@ public class TypedValue {
     }
   }
 
+  private static Object protoSerialToJdbc(Common.Rep type, Object value, Calendar calendar) {
+    switch (type) {
+    case JAVA_UTIL_DATE:
+      return new java.util.Date(adjust((Number) value, calendar));
+    case JAVA_SQL_DATE:
+      return new java.sql.Date(
+          adjust(((Number) value).longValue() * DateTimeUtils.MILLIS_PER_DAY, calendar));
+    case JAVA_SQL_TIME:
+      return new java.sql.Time(adjust((Number) value, calendar));
+    case JAVA_SQL_TIMESTAMP:
+      return new java.sql.Timestamp(adjust((Number) value, calendar));
+    default:
+      return protoSerialToLocal(type, value);
+    }
+  }
+
   private static long adjust(Number number, Calendar calendar) {
     long t = number.longValue();
     if (calendar != null) {
@@ -332,6 +383,10 @@ public class TypedValue {
     return list;
   }
 
+  /**
+   * Creates a protocol buffer equivalent object for <code>this</code>.
+   * @return A protobuf TypedValue equivalent for <code>this</code>
+   */
   public Common.TypedValue toProto() {
     final Common.TypedValue.Builder builder = Common.TypedValue.newBuilder();
 
@@ -419,83 +474,99 @@ public class TypedValue {
     return builder.build();
   }
 
+  /**
+   * Constructs a {@link TypedValue} from the protocol buffer representation.
+   *
+   * @param proto The protobuf Typedvalue
+   * @return A {@link TypedValue} instance
+   */
   public static TypedValue fromProto(Common.TypedValue proto) {
     ColumnMetaData.Rep rep = ColumnMetaData.Rep.fromProto(proto.getType());
+    Object value = getValue(proto);
 
-    Object value = null;
+    return new TypedValue(rep, value);
+  }
 
+  /**
+   * Converts the serialized value into the appropriate primitive/object.
+   *
+   * @param protoValue The serialized TypedValue.
+   * @return The appropriate concrete type for the parameter value (as an Object).
+   */
+  public static Object getValue(Common.TypedValue protoValue) {
     // Deserialize the value again
-    switch (proto.getType()) {
+    switch (protoValue.getType()) {
     case BOOLEAN:
     case PRIMITIVE_BOOLEAN:
-      value = proto.getBoolValue();
-      break;
+      return protoValue.getBoolValue();
     case BYTE_STRING:
     case STRING:
-      value = proto.getStringValue();
-      break;
+      // TypedValue is still going to expect a string for BYTE_STRING even though we sent it
+      // across the wire natively as bytes.
+      return protoValue.getStringValue();
     case PRIMITIVE_CHAR:
     case CHARACTER:
-      value = proto.getStringValue().charAt(0);
-      break;
+      return protoValue.getStringValue().charAt(0);
     case BYTE:
     case PRIMITIVE_BYTE:
-      value = Long.valueOf(proto.getNumberValue()).byteValue();
-      break;
+      return Long.valueOf(protoValue.getNumberValue()).byteValue();
     case DOUBLE:
     case PRIMITIVE_DOUBLE:
-      value = proto.getDoubleValue();
-      break;
+      return protoValue.getDoubleValue();
     case FLOAT:
     case PRIMITIVE_FLOAT:
-      value = Float.intBitsToFloat((int) proto.getNumberValue());
-      break;
+      return Float.intBitsToFloat((int) protoValue.getNumberValue());
     case INTEGER:
     case PRIMITIVE_INT:
-      value = Long.valueOf(proto.getNumberValue()).intValue();
-      break;
+      return Long.valueOf(protoValue.getNumberValue()).intValue();
     case PRIMITIVE_SHORT:
     case SHORT:
-      value = Long.valueOf(proto.getNumberValue()).shortValue();
-      break;
+      return Long.valueOf(protoValue.getNumberValue()).shortValue();
     case LONG:
     case PRIMITIVE_LONG:
-      value = Long.valueOf(proto.getNumberValue());
-      break;
+      return Long.valueOf(protoValue.getNumberValue());
     case JAVA_SQL_DATE:
     case JAVA_SQL_TIME:
-      value = Long.valueOf(proto.getNumberValue()).intValue();
-      break;
+      return Long.valueOf(protoValue.getNumberValue()).intValue();
     case JAVA_SQL_TIMESTAMP:
     case JAVA_UTIL_DATE:
-      value = proto.getNumberValue();
-      break;
+      return protoValue.getNumberValue();
     case BIG_INTEGER:
-      value = new BigInteger(proto.getBytesValues().toByteArray());
-      break;
+      return new BigInteger(protoValue.getBytesValues().toByteArray());
     case BIG_DECIMAL:
-      BigInteger bigInt = new BigInteger(proto.getBytesValues().toByteArray());
-      value = new BigDecimal(bigInt, (int) proto.getNumberValue());
-      break;
+      BigInteger bigInt = new BigInteger(protoValue.getBytesValues().toByteArray());
+      return new BigDecimal(bigInt, (int) protoValue.getNumberValue());
     case NUMBER:
-      value = Long.valueOf(proto.getNumberValue());
-      break;
+      return Long.valueOf(protoValue.getNumberValue());
     case OBJECT:
-      if (proto.getNull()) {
-        value = null;
-        break;
+      if (protoValue.getNull()) {
+        return null;
       }
       // Intentional fall through to RTE. If we sent an object over the wire, it could only
       // possibly be null (at this point). Anything else has to be an error.
     case UNRECOGNIZED:
       // Fail?
-      throw new RuntimeException("Unhandled type: " + proto.getType());
+      throw new RuntimeException("Unhandled type: " + protoValue.getType());
     default:
       // Fail?
-      throw new RuntimeException("Unknown type: " + proto.getType());
+      throw new RuntimeException("Unknown type: " + protoValue.getType());
     }
+  }
 
-    return new TypedValue(rep, value);
+  /**
+   * Extracts the JDBC value from protobuf-TypedValue representation.
+   *
+   * @param protoValue Protobuf TypedValue
+   * @param calendar Instance of a calendar
+   * @return The JDBC representation of this TypedValue
+   */
+  public static Object protoToJdbc(Common.TypedValue protoValue, Calendar calendar) {
+    Object o = getValue(Objects.requireNonNull(protoValue));
+    // Shortcircuit the null
+    if (null == o) {
+      return o;
+    }
+    return protoSerialToJdbc(protoValue.getType(), o, Objects.requireNonNull(calendar));
   }
 
   @Override public int hashCode() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/protobuf/requests.proto
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/protobuf/requests.proto b/avatica/core/src/main/protobuf/requests.proto
index 31b0941..afa8aec 100644
--- a/avatica/core/src/main/protobuf/requests.proto
+++ b/avatica/core/src/main/protobuf/requests.proto
@@ -143,3 +143,21 @@ message CommitRequest {
 message RollbackRequest {
   string connection_id = 1;
 }
+
+// Request to prepare and execute a collection of sql statements.
+message PrepareAndExecuteBatchRequest {
+  string connection_id = 1;
+  uint32 statement_id = 2;
+  repeated string sql_commands = 3;
+}
+
+// Each command is a list of TypedValues
+message UpdateBatch {
+  repeated TypedValue parameter_values = 1;
+}
+
+message ExecuteBatchRequest {
+  string connection_id = 1;
+  uint32 statement_id = 2;
+  repeated UpdateBatch updates = 3; // A batch of updates is a list<list<typevalue>>
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/main/protobuf/responses.proto
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/protobuf/responses.proto b/avatica/core/src/main/protobuf/responses.proto
index 01a62ed..47d73ab 100644
--- a/avatica/core/src/main/protobuf/responses.proto
+++ b/avatica/core/src/main/protobuf/responses.proto
@@ -124,3 +124,12 @@ message CommitResponse {
 message RollbackResponse {
 
 }
+
+// Response to a batch update request
+message ExecuteBatchResponse {
+  string connection_id = 1;
+  uint32 statement_id = 2;
+  repeated uint32 update_counts = 3;
+  bool missing_statement = 4; // Did the request fail because of no-cached statement
+  RpcMetadata metadata = 5;
+}

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ExecuteBatchRequestTest.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ExecuteBatchRequestTest.java b/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ExecuteBatchRequestTest.java
new file mode 100644
index 0000000..134ea15
--- /dev/null
+++ b/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ExecuteBatchRequestTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.proto.Common;
+import org.apache.calcite.avatica.proto.Requests;
+import org.apache.calcite.avatica.proto.Requests.UpdateBatch;
+import org.apache.calcite.avatica.remote.Service.ExecuteBatchRequest;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class for ExecuteBatchRequest
+ */
+public class ExecuteBatchRequestTest {
+
+  private ExecuteBatchRequest identityRequest = new ExecuteBatchRequest();
+  private List<TypedValue> paramValues =
+      Arrays.asList(TypedValue.create(Rep.BOOLEAN.name(), Boolean.TRUE),
+          TypedValue.create(Rep.STRING.name(), "string"));
+
+  @Test public void testConversionFromProtobuf() {
+    ExecuteBatchRequest request = new ExecuteBatchRequest("connectionId", 12345,
+        Arrays.asList(paramValues, paramValues, paramValues));
+
+    assertFalse("A request with the POJO TypedValue list should return false",
+        request.hasProtoUpdateBatches());
+
+    // Everything will be serialized via protobuf
+    Requests.ExecuteBatchRequest protoRequest = request.serialize();
+
+    ExecuteBatchRequest copy = identityRequest.deserialize(protoRequest);
+
+    assertNull("Parameter values (pojo) list should be null", copy.parameterValues);
+    assertTrue("hasProtoUpdateBatches() should return true", copy.hasProtoUpdateBatches());
+    List<UpdateBatch> protoParameterValues = copy.getProtoUpdateBatches();
+    assertNotNull("Protobuf serialized parameterValues should not be null", protoParameterValues);
+
+    assertEquals(request.parameterValues.size(), protoParameterValues.size());
+
+    for (int i = 0; i < request.parameterValues.size(); i++) {
+      List<TypedValue> orig = request.parameterValues.get(i);
+      List<Common.TypedValue> proto = protoParameterValues.get(i).getParameterValuesList();
+      assertEquals("Mismatch in length of TypedValues at index " + i, orig.size(), proto.size());
+
+      // Don't re-test TypedValue serialization
+    }
+
+    // Everything else should be equivalent.
+    assertEquals(request.connectionId, copy.connectionId);
+    assertEquals(request.statementId, copy.statementId);
+  }
+}
+
+// End ExecuteBatchRequestTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java b/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
index c75bdb0..8dac427 100644
--- a/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
+++ b/avatica/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
@@ -44,11 +44,13 @@ import org.apache.calcite.avatica.remote.Service.CreateStatementResponse;
 import org.apache.calcite.avatica.remote.Service.DatabasePropertyRequest;
 import org.apache.calcite.avatica.remote.Service.DatabasePropertyResponse;
 import org.apache.calcite.avatica.remote.Service.ErrorResponse;
+import org.apache.calcite.avatica.remote.Service.ExecuteBatchResponse;
 import org.apache.calcite.avatica.remote.Service.ExecuteResponse;
 import org.apache.calcite.avatica.remote.Service.FetchRequest;
 import org.apache.calcite.avatica.remote.Service.FetchResponse;
 import org.apache.calcite.avatica.remote.Service.OpenConnectionRequest;
 import org.apache.calcite.avatica.remote.Service.OpenConnectionResponse;
+import org.apache.calcite.avatica.remote.Service.PrepareAndExecuteBatchRequest;
 import org.apache.calcite.avatica.remote.Service.PrepareAndExecuteRequest;
 import org.apache.calcite.avatica.remote.Service.PrepareRequest;
 import org.apache.calcite.avatica.remote.Service.PrepareResponse;
@@ -216,6 +218,11 @@ public class ProtobufTranslationImplTest<T> {
     requests.add(new CommitRequest("connectionId"));
     requests.add(new RollbackRequest("connectionId"));
 
+    // ExecuteBatchRequest omitted because of the special protobuf conversion it does
+
+    List<String> commands = Arrays.asList("command1", "command2", "command3");
+    requests.add(new PrepareAndExecuteBatchRequest("connectionId", 12345, commands));
+
     return requests;
   }
 
@@ -351,6 +358,10 @@ public class ProtobufTranslationImplTest<T> {
     responses.add(new CommitResponse());
     responses.add(new RollbackResponse());
 
+    int[] updateCounts = new int[]{1, 0, 1, 1};
+    responses.add(
+        new ExecuteBatchResponse("connectionId", 12345, updateCounts, false, rpcMetadata));
+
     return responses;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/core/src/test/java/org/apache/calcite/avatica/test/JsonHandlerTest.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/test/java/org/apache/calcite/avatica/test/JsonHandlerTest.java b/avatica/core/src/test/java/org/apache/calcite/avatica/test/JsonHandlerTest.java
index 57cf60a..7afa000 100644
--- a/avatica/core/src/test/java/org/apache/calcite/avatica/test/JsonHandlerTest.java
+++ b/avatica/core/src/test/java/org/apache/calcite/avatica/test/JsonHandlerTest.java
@@ -129,6 +129,14 @@ public class JsonHandlerTest {
     @Override public RollbackResponse apply(RollbackRequest request) {
       return null;
     }
+
+    @Override public ExecuteBatchResponse apply(ExecuteBatchRequest request) {
+      return null;
+    }
+
+    @Override public ExecuteBatchResponse apply(PrepareAndExecuteBatchRequest request) {
+      return null;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/5dfa3f1e/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index dfe7f99..4e6c67f 100644
--- a/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -31,6 +31,9 @@ import org.apache.calcite.avatica.SqlType;
 import org.apache.calcite.avatica.metrics.Gauge;
 import org.apache.calcite.avatica.metrics.MetricsSystem;
 import org.apache.calcite.avatica.metrics.noop.NoopMetricsSystem;
+import org.apache.calcite.avatica.proto.Common;
+import org.apache.calcite.avatica.proto.Requests;
+import org.apache.calcite.avatica.remote.ProtobufMeta;
 import org.apache.calcite.avatica.remote.TypedValue;
 
 import com.google.common.cache.Cache;
@@ -66,7 +69,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 /** Implementation of {@link Meta} upon an existing JDBC data source. */
-public class JdbcMeta implements Meta {
+public class JdbcMeta implements ProtobufMeta {
   private static final Logger LOG = LoggerFactory.getLogger(JdbcMeta.class);
 
   private static final String CONN_CACHE_KEY_BASE = "avatica.connectioncache";
@@ -852,6 +855,83 @@ public class JdbcMeta implements Meta {
     }
   }
 
+  @Override public ExecuteBatchResult prepareAndExecuteBatch(StatementHandle h,
+      List<String> sqlCommands) throws NoSuchStatementException {
+    try {
+      // Get the statement
+      final StatementInfo info = statementCache.getIfPresent(h.id);
+      if (info == null) {
+        throw new NoSuchStatementException(h);
+      }
+
+      // addBatch() for each sql command
+      final Statement stmt = info.statement;
+      for (String sqlCommand : sqlCommands) {
+        stmt.addBatch(sqlCommand);
+      }
+
+      // Execute the batch and return the results
+      return new ExecuteBatchResult(stmt.executeBatch());
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
+  @Override public ExecuteBatchResult executeBatch(StatementHandle h,
+      List<List<TypedValue>> updateBatches) throws NoSuchStatementException {
+    try {
+      final StatementInfo info = statementCache.getIfPresent(h.id);
+      if (null == info) {
+        throw new NoSuchStatementException(h);
+      }
+
+      final PreparedStatement preparedStmt = (PreparedStatement) info.statement;
+      int rowUpdate = 1;
+      for (List<TypedValue> batch : updateBatches) {
+        int i = 1;
+        for (TypedValue value : batch) {
+          // Set the TypedValue in the PreparedStatement
+          try {
+            preparedStmt.setObject(i, value.toJdbc(calendar));
+            i++;
+          } catch (SQLException e) {
+            throw new RuntimeException("Failed to set value on row #" + rowUpdate
+                + " and column #" + i, e);
+          }
+          // Track the update number for better error messages
+          rowUpdate++;
+        }
+        preparedStmt.addBatch();
+      }
+      return new ExecuteBatchResult(preparedStmt.executeBatch());
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
+  @Override public ExecuteBatchResult executeBatchProtobuf(StatementHandle h,
+      List<Requests.UpdateBatch> updateBatches) throws NoSuchStatementException {
+    try {
+      final StatementInfo info = statementCache.getIfPresent(h.id);
+      if (null == info) {
+        throw new NoSuchStatementException(h);
+      }
+
+      final PreparedStatement preparedStmt = (PreparedStatement) info.statement;
+      for (Requests.UpdateBatch update : updateBatches) {
+        int i = 1;
+        for (Common.TypedValue value : update.getParameterValuesList()) {
+          // Use the value and then increment
+          preparedStmt.setObject(i++, TypedValue.protoToJdbc(value, calendar));
+        }
+        preparedStmt.addBatch();
+      }
+      return new ExecuteBatchResult(preparedStmt.executeBatch());
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
   /** Configurable statement cache settings. */
   public enum StatementCacheSettings {
     /** JDBC connection property for setting connection cache concurrency level. */