You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/07/24 10:23:17 UTC

[GitHub] [flink] leonardBang commented on a change in pull request #12882: [FLINK-18579][FLINK-15728][jdbc] Introduce FieldNamedPreparedStatement to support fields are bound multiple times in update statement

leonardBang commented on a change in pull request #12882:
URL: https://github.com/apache/flink/pull/12882#discussion_r459940712



##########
File path: flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/statement/FieldNamedPreparedStatementImplTest.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.flink.connector.jdbc.statement;
+
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link FieldNamedPreparedStatementImpl}.
+ */
+public class FieldNamedPreparedStatementImplTest {
+
+	private final JdbcDialect dialect = JdbcDialects.get("jdbc:mysql://localhost:3306/test")
+		.orElseThrow(() -> new RuntimeException("Unsupported dialect."));
+	private final String[] fieldNames = new String[]{"id", "name", "email", "ts", "field1", "field_2", "__field_3__"};
+	private final String[] keyFields = new String[]{"id", "__field_3__"};
+	private final String tableName = "tbl";
+
+	@Test
+	public void testInsertStatement() {
+		String insertStmt = dialect.getInsertIntoStatement(tableName, fieldNames);
+		assertEquals(
+			"INSERT INTO `tbl`(`id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__`) " +
+				"VALUES (:id, :name, :email, :ts, :field1, :field_2, :__field_3__)",
+			insertStmt);
+		NamedStatementMatcher
+			.parsedSql("INSERT INTO `tbl`(`id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__`) " +
+				"VALUES (?, ?, ?, ?, ?, ?, ?)")
+			.parameter("id", singletonList(1))
+			.parameter("name", singletonList(2))
+			.parameter("email", singletonList(3))
+			.parameter("ts", singletonList(4))
+			.parameter("field1", singletonList(5))
+			.parameter("field_2", singletonList(6))
+			.parameter("__field_3__", singletonList(7))
+			.matches(insertStmt);
+	}
+
+	@Test
+	public void testDeleteStatement() {
+		String deleteStmt = dialect.getDeleteStatement(tableName, keyFields);
+		assertEquals(
+			"DELETE FROM `tbl` WHERE `id` = :id AND `__field_3__` = :__field_3__",
+			deleteStmt);
+		NamedStatementMatcher
+			.parsedSql("DELETE FROM `tbl` WHERE `id` = ? AND `__field_3__` = ?")
+			.parameter("id", singletonList(1))
+			.parameter("__field_3__", singletonList(2))
+			.matches(deleteStmt);
+	}
+
+	@Test
+	public void testRowExistsStatement() {
+		String rowExistStmt = dialect.getRowExistsStatement(tableName, keyFields);
+		assertEquals(
+			"SELECT 1 FROM `tbl` WHERE `id` = :id AND `__field_3__` = :__field_3__",
+			rowExistStmt);
+		NamedStatementMatcher
+			.parsedSql("SELECT 1 FROM `tbl` WHERE `id` = ? AND `__field_3__` = ?")
+			.parameter("id", singletonList(1))
+			.parameter("__field_3__", singletonList(2))
+			.matches(rowExistStmt);
+	}
+
+	@Test
+	public void testUpdateStatement() {
+		String updateStmt = dialect.getUpdateStatement(tableName, fieldNames, keyFields);
+		assertEquals(
+			"UPDATE `tbl` SET `id` = :id, `name` = :name, `email` = :email, `ts` = :ts, " +
+				"`field1` = :field1, `field_2` = :field_2, `__field_3__` = :__field_3__ " +
+				"WHERE `id` = :id AND `__field_3__` = :__field_3__",
+			updateStmt);
+		NamedStatementMatcher
+			.parsedSql("UPDATE `tbl` SET `id` = ?, `name` = ?, `email` = ?, `ts` = ?, `field1` = ?, " +
+				"`field_2` = ?, `__field_3__` = ? WHERE `id` = ? AND `__field_3__` = ?")
+			.parameter("id", asList(1, 8))
+			.parameter("name", singletonList(2))
+			.parameter("email", singletonList(3))
+			.parameter("ts", singletonList(4))
+			.parameter("field1", singletonList(5))
+			.parameter("field_2", singletonList(6))
+			.parameter("__field_3__", asList(7, 9))
+			.matches(updateStmt);
+	}
+
+	@Test
+	public void testUpsertStatement() {
+		String upsertStmt = dialect.getUpsertStatement(tableName, fieldNames, keyFields).get();
+		assertEquals(
+			"INSERT INTO `tbl`(`id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__`) " +
+				"VALUES (:id, :name, :email, :ts, :field1, :field_2, :__field_3__) " +
+				"ON DUPLICATE KEY UPDATE `id`=VALUES(`id`), `name`=VALUES(`name`), " +
+				"`email`=VALUES(`email`), `ts`=VALUES(`ts`), `field1`=VALUES(`field1`)," +
+				" `field_2`=VALUES(`field_2`), `__field_3__`=VALUES(`__field_3__`)",
+			upsertStmt);
+		NamedStatementMatcher
+			.parsedSql("INSERT INTO `tbl`(`id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__`) " +
+				"VALUES (?, ?, ?, ?, ?, ?, ?) ON DUPLICATE KEY UPDATE " +
+				"`id`=VALUES(`id`), `name`=VALUES(`name`), `email`=VALUES(`email`), `ts`=VALUES(`ts`)," +
+				" `field1`=VALUES(`field1`), `field_2`=VALUES(`field_2`), `__field_3__`=VALUES(`__field_3__`)")
+			.parameter("id", singletonList(1))
+			.parameter("name", singletonList(2))
+			.parameter("email", singletonList(3))
+			.parameter("ts", singletonList(4))
+			.parameter("field1", singletonList(5))
+			.parameter("field_2", singletonList(6))
+			.parameter("__field_3__", singletonList(7))
+			.matches(upsertStmt);
+	}
+
+	@Test
+	public void testSelectStatement() {
+		String selectStmt = dialect.getSelectFromStatement(tableName, fieldNames, keyFields);
+		assertEquals(
+			"SELECT `id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__` FROM `tbl` " +
+				"WHERE `id` = :id AND `__field_3__` = :__field_3__",
+			selectStmt);
+		NamedStatementMatcher
+			.parsedSql("SELECT `id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__` FROM `tbl` " +
+				"WHERE `id` = ? AND `__field_3__` = ?")
+			.parameter("id", singletonList(1))
+			.parameter("__field_3__", singletonList(2))
+			.matches(selectStmt);
+	}
+
+	private static class NamedStatementMatcher {
+		String parsedSql;

Review comment:
       nit: can be private

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/executor/InsertOrUpdateJdbcExecutor.java
##########
@@ -39,6 +39,9 @@
 /**
  * {@link JdbcBatchStatementExecutor} that provides upsert semantics by updating row if it exists and inserting otherwise.
  * Used in Table API.
+ *
+ * @deprecated this has be replaced with {@link TableInsertOrUpdateStatementExecutor}, will remove

Review comment:
       ```suggestion
    * @deprecated This has been replaced with {@link TableInsertOrUpdateStatementExecutor}, will remove
   ```

##########
File path: flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/statement/FieldNamedPreparedStatementImplTest.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.flink.connector.jdbc.statement;
+
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link FieldNamedPreparedStatementImpl}.
+ */
+public class FieldNamedPreparedStatementImplTest {
+
+	private final JdbcDialect dialect = JdbcDialects.get("jdbc:mysql://localhost:3306/test")

Review comment:
       We can add @parameterized test to cover all dialects.

##########
File path: flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/statement/FieldNamedPreparedStatementImplTest.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.flink.connector.jdbc.statement;
+
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link FieldNamedPreparedStatementImpl}.
+ */
+public class FieldNamedPreparedStatementImplTest {
+
+	private final JdbcDialect dialect = JdbcDialects.get("jdbc:mysql://localhost:3306/test")
+		.orElseThrow(() -> new RuntimeException("Unsupported dialect."));
+	private final String[] fieldNames = new String[]{"id", "name", "email", "ts", "field1", "field_2", "__field_3__"};
+	private final String[] keyFields = new String[]{"id", "__field_3__"};
+	private final String tableName = "tbl";
+
+	@Test
+	public void testInsertStatement() {
+		String insertStmt = dialect.getInsertIntoStatement(tableName, fieldNames);
+		assertEquals(
+			"INSERT INTO `tbl`(`id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__`) " +
+				"VALUES (:id, :name, :email, :ts, :field1, :field_2, :__field_3__)",
+			insertStmt);
+		NamedStatementMatcher
+			.parsedSql("INSERT INTO `tbl`(`id`, `name`, `email`, `ts`, `field1`, `field_2`, `__field_3__`) " +
+				"VALUES (?, ?, ?, ?, ?, ?, ?)")
+			.parameter("id", singletonList(1))
+			.parameter("name", singletonList(2))
+			.parameter("email", singletonList(3))
+			.parameter("ts", singletonList(4))
+			.parameter("field1", singletonList(5))
+			.parameter("field_2", singletonList(6))
+			.parameter("__field_3__", singletonList(7))
+			.matches(insertStmt);
+	}
+
+	@Test
+	public void testDeleteStatement() {
+		String deleteStmt = dialect.getDeleteStatement(tableName, keyFields);
+		assertEquals(
+			"DELETE FROM `tbl` WHERE `id` = :id AND `__field_3__` = :__field_3__",
+			deleteStmt);
+		NamedStatementMatcher
+			.parsedSql("DELETE FROM `tbl` WHERE `id` = ? AND `__field_3__` = ?")
+			.parameter("id", singletonList(1))
+			.parameter("__field_3__", singletonList(2))
+			.matches(deleteStmt);
+	}
+
+	@Test
+	public void testRowExistsStatement() {
+		String rowExistStmt = dialect.getRowExistsStatement(tableName, keyFields);
+		assertEquals(
+			"SELECT 1 FROM `tbl` WHERE `id` = :id AND `__field_3__` = :__field_3__",
+			rowExistStmt);
+		NamedStatementMatcher
+			.parsedSql("SELECT 1 FROM `tbl` WHERE `id` = ? AND `__field_3__` = ?")
+			.parameter("id", singletonList(1))
+			.parameter("__field_3__", singletonList(2))
+			.matches(rowExistStmt);
+	}
+
+	@Test
+	public void testUpdateStatement() {
+		String updateStmt = dialect.getUpdateStatement(tableName, fieldNames, keyFields);
+		assertEquals(
+			"UPDATE `tbl` SET `id` = :id, `name` = :name, `email` = :email, `ts` = :ts, " +
+				"`field1` = :field1, `field_2` = :field_2, `__field_3__` = :__field_3__ " +
+				"WHERE `id` = :id AND `__field_3__` = :__field_3__",
+			updateStmt);
+		NamedStatementMatcher
+			.parsedSql("UPDATE `tbl` SET `id` = ?, `name` = ?, `email` = ?, `ts` = ?, `field1` = ?, " +
+				"`field_2` = ?, `__field_3__` = ? WHERE `id` = ? AND `__field_3__` = ?")
+			.parameter("id", asList(1, 8))
+			.parameter("name", singletonList(2))
+			.parameter("email", singletonList(3))
+			.parameter("ts", singletonList(4))
+			.parameter("field1", singletonList(5))
+			.parameter("field_2", singletonList(6))
+			.parameter("__field_3__", asList(7, 9))
+			.matches(updateStmt);
+	}
+
+	@Test
+	public void testUpsertStatement() {

Review comment:
       Add test for pg dialect

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/executor/TableBufferedStatementExecutor.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.flink.connector.jdbc.internal.executor;
+
+import org.apache.flink.table.data.RowData;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Currently, this statement executor is only used for table/sql to buffer records,
+ * because the {@link PreparedStatement#executeBatch()} may fail and clear added batched,

Review comment:
       nit: added batched -> buffed records 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java
##########
@@ -97,7 +99,7 @@ default String quoteIdentifier(String identifier) {
 	 */
 	default String getRowExistsStatement(String tableName, String[] conditionFields) {
 		String fieldExpressions = Arrays.stream(conditionFields)
-			.map(f -> quoteIdentifier(f) + "=?")
+			.map(f -> format("%s = :%s", quoteIdentifier(f), f))

Review comment:
       Could we also use namedFiled for getUpsertStatement() ?

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/statement/FieldNamedPreparedStatement.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.flink.connector.jdbc.statement;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+/**
+ * This is a wrapper around {@link PreparedStatement} and allows the users to set parameters
+ * by name instead of by index. This allows users to use the same variable parameter multiple
+ * times in a statement.
+ *
+ * <p>Code such as this:
+ *
+ * <pre>
+ *   Connection con = getConnection();
+ *   String query = "select * from my_table where first_name=? or last_name=?";
+ *   PreparedStatement st = con.prepareStatement(query);
+ *   st.setString(1, "bob");
+ *   st.setString(2, "bob");
+ *   ResultSet rs = st.executeQuery();
+ * </pre>
+ *
+ * <p>Can be replaced with:
+ *
+ * <pre>
+ *   Connection con = getConnection();
+ *   String query = "select * from my_table where first_name=:name or last_name=:name";
+ *   FieldNamedPreparedStatement st = FieldNamedPreparedStatement.prepareStatement(con, query, new String[]{"name"});
+ *   st.setString(0, "bob");
+ *   ResultSet rs = st.executeQuery();
+ * </pre>
+ */
+public interface FieldNamedPreparedStatement extends AutoCloseable {

Review comment:
       How about `FieldNamedPreparedStatement extends PreparedStatement` rather than a rapper? in this way we can 
   visit all methods of FieldNamedPreparedStatement from FieldNamedPreparedStatement.
   

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/statement/FieldNamedPreparedStatement.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.flink.connector.jdbc.statement;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+/**
+ * This is a wrapper around {@link PreparedStatement} and allows the users to set parameters
+ * by name instead of by index. This allows users to use the same variable parameter multiple
+ * times in a statement.
+ *
+ * <p>Code such as this:
+ *
+ * <pre>
+ *   Connection con = getConnection();
+ *   String query = "select * from my_table where first_name=? or last_name=?";
+ *   PreparedStatement st = con.prepareStatement(query);
+ *   st.setString(1, "bob");
+ *   st.setString(2, "bob");
+ *   ResultSet rs = st.executeQuery();
+ * </pre>
+ *
+ * <p>Can be replaced with:
+ *
+ * <pre>
+ *   Connection con = getConnection();
+ *   String query = "select * from my_table where first_name=:name or last_name=:name";
+ *   FieldNamedPreparedStatement st = FieldNamedPreparedStatement.prepareStatement(con, query, new String[]{"name"});
+ *   st.setString(0, "bob");
+ *   ResultSet rs = st.executeQuery();
+ * </pre>
+ */
+public interface FieldNamedPreparedStatement extends AutoCloseable {
+
+	/**
+	 * Creates a <code>NamedPreparedStatement</code> object for sending
+	 * parameterized SQL statements to the database.
+	 * @param connection the connection used to connect to database.
+	 * @param sql an SQL statement that may contain one or more ':fieldName' as parameter placeholders
+	 * @param fieldNames the field names in schema order used as the parameter names
+	 */
+	static FieldNamedPreparedStatement prepareStatement(
+			Connection connection, String sql, String[] fieldNames) throws SQLException {
+		return FieldNamedPreparedStatementImpl.prepareStatement(connection, sql, fieldNames);
+	}
+
+	/**
+	 * Clears the current parameter values immediately.
+	 *
+	 * <P>In general, parameter values remain in force for repeated use of a
+	 * statement. Setting a parameter value automatically clears its
+	 * previous value.  However, in some cases it is useful to immediately
+	 * release the resources used by the current parameter values; this can
+	 * be done by calling the method <code>clearParameters</code>.
+	 *
+	 * @see PreparedStatement#clearParameters()
+	 */
+	void clearParameters() throws SQLException;
+
+	/**
+	 * Executes the SQL query in this <code>NamedPreparedStatement</code> object
+	 * and returns the <code>ResultSet</code> object generated by the query.
+	 *
+	 * @see PreparedStatement#executeQuery()
+	 */
+	ResultSet executeQuery() throws SQLException;
+
+	/**
+	 * Adds a set of parameters to this <code>NamedPreparedStatement</code>
+	 * object's batch of commands.
+	 *
+	 * @see PreparedStatement#addBatch()
+	 */
+	void addBatch() throws SQLException;
+
+	/**
+	 * Submits a batch of commands to the database for execution and
+	 * if all commands execute successfully, returns an array of update counts.
+	 * The <code>int</code> elements of the array that is returned are ordered
+	 * to correspond to the commands in the batch, which are ordered
+	 * according to the order in which they were added to the batch.
+	 *
+	 * @see PreparedStatement#executeBatch()
+	 */
+	int[] executeBatch() throws SQLException;
+
+	/**
+	 * Sets the designated parameter to SQL <code>NULL</code>.
+	 *
+	 * <P><B>Note:</B> You must specify the parameter's SQL type.
+	 *
+	 * @see PreparedStatement#setNull(int, int)
+	 */
+	void setNull(int fieldIndex, int sqlType) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>boolean</code> value.
+	 * The driver converts this
+	 * to an SQL <code>BIT</code> or <code>BOOLEAN</code> value when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setBoolean(int, boolean)
+	 */
+	void setBoolean(int fieldIndex, boolean x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>byte</code> value.
+	 * The driver converts this
+	 * to an SQL <code>TINYINT</code> value when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setByte(int, byte)
+	 */
+	void setByte(int fieldIndex, byte x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>short</code> value.
+	 * The driver converts this
+	 * to an SQL <code>SMALLINT</code> value when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setShort(int, short)
+	 */
+	void setShort(int fieldIndex, short x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>int</code> value.
+	 * The driver converts this
+	 * to an SQL <code>INTEGER</code> value when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setInt(int, int)
+	 */
+	void setInt(int fieldIndex, int x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>long</code> value.
+	 * The driver converts this
+	 * to an SQL <code>BIGINT</code> value when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setLong(int, long)
+	 */
+	void setLong(int fieldIndex, long x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>float</code> value.
+	 * The driver converts this
+	 * to an SQL <code>REAL</code> value when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setFloat(int, float)
+	 */
+	void setFloat(int fieldIndex, float x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>double</code> value.
+	 * The driver converts this
+	 * to an SQL <code>DOUBLE</code> value when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setDouble(int, double)
+	 */
+	void setDouble(int fieldIndex, double x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given <code>java.math.BigDecimal</code> value.
+	 * The driver converts this to an SQL <code>NUMERIC</code> value when
+	 * it sends it to the database.
+	 *
+	 * @see PreparedStatement#setBigDecimal(int, BigDecimal)
+	 */
+	void setBigDecimal(int fieldIndex, BigDecimal x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java <code>String</code> value.
+	 * The driver converts this
+	 * to an SQL <code>VARCHAR</code> or <code>LONGVARCHAR</code> value
+	 * (depending on the argument's
+	 * size relative to the driver's limits on <code>VARCHAR</code> values)
+	 * when it sends it to the database.
+	 *
+	 * @see PreparedStatement#setString(int, String)
+	 */
+	void setString(int fieldIndex, String x) throws SQLException;
+
+	/**
+	 * Sets the designated parameter to the given Java array of bytes.  The driver converts

Review comment:
       tip: double spaces




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org