From d99fbcbdb417669e5bec9307b77eb6a113cf7657 Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Thu, 25 Dec 2025 14:36:49 +0800 Subject: [PATCH 1/8] fix: typo in CreatePipe --- .../db/queryengine/plan/relational/sql/ast/CreatePipe.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/CreatePipe.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/CreatePipe.java index 1543e339fc86..269978e87bde 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/CreatePipe.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/CreatePipe.java @@ -115,9 +115,9 @@ public long ramBytesUsed() { long size = INSTANCE_SIZE; size += AstMemoryEstimationHelper.getEstimatedSizeOfNodeLocation(getLocationInternal()); size += RamUsageEstimator.sizeOf(pipeName); - size += RamUsageEstimator.sizeOfMap(extractorAttributes); + size += RamUsageEstimator.sizeOfMap(sourceAttributes); size += RamUsageEstimator.sizeOfMap(processorAttributes); - size += RamUsageEstimator.sizeOfMap(connectorAttributes); + size += RamUsageEstimator.sizeOfMap(sinkAttributes); return size; } } From b065532763619d97f42fc83d6ce4d09ead11dc48 Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Wed, 14 Jan 2026 18:37:52 +0800 Subject: [PATCH 2/8] feat(jdbc): implement PreparedStatement --- .../iotdb/jdbc/IoTDBPreparedStatement.java | 1118 +++++++---------- .../org/apache/iotdb/jdbc/IoTDBStatement.java | 6 +- .../jdbc/IoTDBPreparedStatementTest.java | 311 ++--- .../thrift/impl/ClientRPCServiceImpl.java | 233 ++++ .../db/queryengine/plan/Coordinator.java | 53 +- .../src/main/thrift/client.thrift | 37 + 6 files changed, 917 insertions(+), 841 deletions(-) diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java index c92b6549bf9d..032b9769df36 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java @@ -19,7 +19,16 @@ package org.apache.iotdb.jdbc; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.StatementExecutionException; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.IClientRPCService.Iface; +import org.apache.iotdb.service.rpc.thrift.TSDeallocatePreparedReq; +import org.apache.iotdb.service.rpc.thrift.TSExecutePreparedReq; +import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp; +import org.apache.iotdb.service.rpc.thrift.TSPrepareReq; +import org.apache.iotdb.service.rpc.thrift.TSPrepareResp; import org.apache.thrift.TException; import org.apache.tsfile.common.conf.TSFileConfig; @@ -31,10 +40,11 @@ import java.io.IOException; import java.io.InputStream; import java.io.Reader; -import java.io.StringReader; import java.math.BigDecimal; import java.net.URL; +import java.nio.ByteBuffer; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.sql.Array; import java.sql.Blob; import java.sql.Clob; @@ -52,7 +62,6 @@ import java.sql.Timestamp; import java.sql.Types; import java.text.DateFormat; -import java.text.ParsePosition; import java.text.SimpleDateFormat; import java.time.Instant; import java.time.ZoneId; @@ -62,16 +71,31 @@ import java.util.Calendar; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; +import java.util.UUID; public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedStatement { - private String sql; - private static final String METHOD_NOT_SUPPORTED_STRING = "Method not supported"; private static final Logger logger = LoggerFactory.getLogger(IoTDBPreparedStatement.class); + private static final String METHOD_NOT_SUPPORTED_STRING = "Method not supported"; + + private final String sql; + private final String preparedStatementName; + private final int parameterCount; + + // Parameter values stored as objects for binary serialization + private final Object[] parameterValues; + private final int[] parameterTypes; - /** save the SQL parameters as (paramLoc,paramValue) pairs. */ + // Parameter type constants for serialization + private static final byte TYPE_NULL = 0x00; + private static final byte TYPE_BOOLEAN = 0x01; + private static final byte TYPE_LONG = 0x02; + private static final byte TYPE_DOUBLE = 0x03; + private static final byte TYPE_STRING = 0x04; + private static final byte TYPE_BINARY = 0x05; + + /** save the SQL parameters as (paramLoc,paramValue) pairs for backward compatibility. */ private final Map parameters = new HashMap<>(); IoTDBPreparedStatement( @@ -84,14 +108,42 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt throws SQLException { super(connection, client, sessionId, zoneId, charset); this.sql = sql; + this.preparedStatementName = generateStatementName(); + + // Send PREPARE request to server + TSPrepareReq prepareReq = new TSPrepareReq(); + prepareReq.setSessionId(sessionId); + prepareReq.setSql(sql); + prepareReq.setStatementName(preparedStatementName); + + try { + TSPrepareResp resp = client.prepareStatement(prepareReq); + RpcUtils.verifySuccess(resp.getStatus()); + + this.parameterCount = resp.isSetParameterCount() ? resp.getParameterCount() : 0; + this.parameterValues = new Object[parameterCount]; + this.parameterTypes = new int[parameterCount]; + + // Initialize all parameter types to NULL + for (int i = 0; i < parameterCount; i++) { + parameterTypes[i] = Types.NULL; + } + } catch (TException e) { + throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); + } catch (StatementExecutionException e) { + throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); + } } // Only for tests IoTDBPreparedStatement( IoTDBConnection connection, Iface client, Long sessionId, String sql, ZoneId zoneId) throws SQLException { - super(connection, client, sessionId, zoneId, TSFileConfig.STRING_CHARSET); - this.sql = sql; + this(connection, client, sessionId, sql, zoneId, TSFileConfig.STRING_CHARSET); + } + + private String generateStatementName() { + return "jdbc_ps_" + UUID.randomUUID().toString().replace("-", ""); } @Override @@ -102,26 +154,186 @@ public void addBatch() throws SQLException { @Override public void clearParameters() { this.parameters.clear(); + for (int i = 0; i < parameterCount; i++) { + parameterValues[i] = null; + parameterTypes[i] = Types.NULL; + } } @Override public boolean execute() throws SQLException { - return super.execute(createCompleteSql(sql, parameters)); + TSExecuteStatementResp resp = executeInternal(); + return resp.isSetQueryDataSet() || resp.isSetQueryResult(); } @Override public ResultSet executeQuery() throws SQLException { - return super.executeQuery(createCompleteSql(sql, parameters)); + TSExecuteStatementResp resp = executeInternal(); + return processQueryResult(resp); } @Override public int executeUpdate() throws SQLException { - return super.executeUpdate(createCompleteSql(sql, parameters)); + executeInternal(); + return 0; // IoTDB doesn't return affected row count + } + + private TSExecuteStatementResp executeInternal() throws SQLException { + // Validate all parameters are set + for (int i = 0; i < parameterCount; i++) { + if (parameterTypes[i] == Types.NULL + && parameterValues[i] == null + && !parameters.containsKey(i + 1)) { + throw new SQLException("Parameter #" + (i + 1) + " is unset"); + } + } + + TSExecutePreparedReq req = new TSExecutePreparedReq(); + req.setSessionId(sessionId); + req.setStatementName(preparedStatementName); + req.setParameters(serializeParameters()); + + if (queryTimeout > 0) { + req.setTimeout(queryTimeout * 1000L); + } + + try { + TSExecuteStatementResp resp = client.executePreparedStatement(req); + RpcUtils.verifySuccess(resp.getStatus()); + return resp; + } catch (TException e) { + throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); + } catch (StatementExecutionException e) { + throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); + } + } + + private ResultSet processQueryResult(TSExecuteStatementResp resp) throws SQLException { + if (resp.isSetQueryDataSet() || resp.isSetQueryResult()) { + // Create ResultSet from response + this.resultSet = + new IoTDBJDBCResultSet( + this, + resp.getColumns(), + resp.getDataTypeList(), + resp.columnNameIndexMap, + resp.ignoreTimeStamp, + client, + sql, + resp.queryId, + sessionId, + resp.queryResult, + resp.tracingInfo, + (long) queryTimeout * 1000, + resp.isSetMoreData() && resp.isMoreData(), + zoneId); + return resultSet; + } + return null; + } + + /** + * Serialize parameters to binary format for transmission. Format: [type:1byte][value:variable] + */ + private List serializeParameters() { + List serialized = new ArrayList<>(); + for (int i = 0; i < parameterCount; i++) { + serialized.add(serializeParameter(i)); + } + return serialized; + } + + private ByteBuffer serializeParameter(int index) { + Object value = parameterValues[index]; + int type = parameterTypes[index]; + + if (value == null || type == Types.NULL) { + return ByteBuffer.wrap(new byte[] {TYPE_NULL}); + } + + switch (type) { + case Types.BOOLEAN: + ByteBuffer boolBuf = ByteBuffer.allocate(2); + boolBuf.put(TYPE_BOOLEAN); + boolBuf.put((byte) ((Boolean) value ? 1 : 0)); + boolBuf.flip(); + return boolBuf; + + case Types.INTEGER: + case Types.BIGINT: + ByteBuffer longBuf = ByteBuffer.allocate(9); + longBuf.put(TYPE_LONG); + longBuf.putLong(((Number) value).longValue()); + longBuf.flip(); + return longBuf; + + case Types.FLOAT: + case Types.DOUBLE: + ByteBuffer doubleBuf = ByteBuffer.allocate(9); + doubleBuf.put(TYPE_DOUBLE); + doubleBuf.putDouble(((Number) value).doubleValue()); + doubleBuf.flip(); + return doubleBuf; + + case Types.VARCHAR: + case Types.CHAR: + byte[] strBytes = ((String) value).getBytes(StandardCharsets.UTF_8); + ByteBuffer strBuf = ByteBuffer.allocate(5 + strBytes.length); + strBuf.put(TYPE_STRING); + strBuf.putInt(strBytes.length); + strBuf.put(strBytes); + strBuf.flip(); + return strBuf; + + case Types.BINARY: + case Types.VARBINARY: + byte[] binBytes = (byte[]) value; + ByteBuffer binBuf = ByteBuffer.allocate(5 + binBytes.length); + binBuf.put(TYPE_BINARY); + binBuf.putInt(binBytes.length); + binBuf.put(binBytes); + binBuf.flip(); + return binBuf; + + default: + // Fallback: serialize as string + String strValue = String.valueOf(value); + byte[] defaultBytes = strValue.getBytes(StandardCharsets.UTF_8); + ByteBuffer defaultBuf = ByteBuffer.allocate(5 + defaultBytes.length); + defaultBuf.put(TYPE_STRING); + defaultBuf.putInt(defaultBytes.length); + defaultBuf.put(defaultBytes); + defaultBuf.flip(); + return defaultBuf; + } + } + + @Override + public void close() throws SQLException { + if (!isClosed()) { + // Deallocate prepared statement on server + TSDeallocatePreparedReq req = new TSDeallocatePreparedReq(); + req.setSessionId(sessionId); + req.setStatementName(preparedStatementName); + + try { + TSStatus status = client.deallocatePreparedStatement(req); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + logger.warn("Failed to deallocate prepared statement: {}", status.getMessage()); + } + } catch (TException e) { + logger.warn("Error deallocating prepared statement", e); + } + } + super.close(); } @Override public ResultSetMetaData getMetaData() throws SQLException { - return getResultSet().getMetaData(); + if (resultSet != null) { + return resultSet.getMetaData(); + } + return null; } @Override @@ -129,7 +341,7 @@ public ParameterMetaData getParameterMetaData() { return new ParameterMetaData() { @Override public int getParameterCount() { - return parameters.size(); + return parameterCount; } @Override @@ -139,43 +351,26 @@ public int isNullable(int param) { @Override public boolean isSigned(int param) { - try { - return Integer.parseInt(parameters.get(param)) < 0; - } catch (Exception e) { - return false; - } + int type = parameterTypes[param - 1]; + return type == Types.INTEGER + || type == Types.BIGINT + || type == Types.FLOAT + || type == Types.DOUBLE; } @Override public int getPrecision(int param) { - return parameters.get(param).length(); + return 0; } @Override public int getScale(int param) { - try { - double d = Double.parseDouble(parameters.get(param)); - if (d >= 1) { // we only need the fraction digits - d = d - (long) d; - } - if (d == 0) { // nothing to count - return 0; - } - d *= 10; // shifts 1 digit to left - int count = 1; - while (d - (long) d != 0) { // keeps shifting until there are no more fractions - d *= 10; - count++; - } - return count; - } catch (Exception e) { - return 0; - } + return 0; } @Override public int getParameterType(int param) { - return 0; + return parameterTypes[param - 1]; } @Override @@ -190,7 +385,7 @@ public String getParameterClassName(int param) { @Override public int getParameterMode(int param) { - return 0; + return ParameterMetaData.parameterModeIn; } @Override @@ -205,799 +400,347 @@ public boolean isWrapperFor(Class iface) { }; } + // ================== Parameter Setters ================== + @Override - public void setArray(int parameterIndex, Array x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setNull(int parameterIndex, int sqlType) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = null; + parameterTypes[parameterIndex - 1] = Types.NULL; + this.parameters.put(parameterIndex, "NULL"); } @Override - public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException { + setNull(parameterIndex, sqlType); } @Override - public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setBoolean(int parameterIndex, boolean x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.BOOLEAN; + this.parameters.put(parameterIndex, Boolean.toString(x)); } @Override - public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setInt(int parameterIndex, int x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = (long) x; + parameterTypes[parameterIndex - 1] = Types.INTEGER; + this.parameters.put(parameterIndex, Integer.toString(x)); } @Override - public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setLong(int parameterIndex, long x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.BIGINT; + this.parameters.put(parameterIndex, Long.toString(x)); } @Override - public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setFloat(int parameterIndex, float x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = (double) x; + parameterTypes[parameterIndex - 1] = Types.FLOAT; + this.parameters.put(parameterIndex, Float.toString(x)); } @Override - public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { - byte[] bytes = null; - try { - bytes = ReadWriteIOUtils.readBytes(x, length); - StringBuilder sb = new StringBuilder(); - for (byte b : bytes) { - sb.append(String.format("%02x", b)); - } - this.parameters.put(parameterIndex, "X'" + sb.toString() + "'"); - } catch (IOException e) { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setDouble(int parameterIndex, double x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.DOUBLE; + this.parameters.put(parameterIndex, Double.toString(x)); + } + + @Override + public void setString(int parameterIndex, String x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.VARCHAR; + if (x == null) { + this.parameters.put(parameterIndex, null); + } else { + this.parameters.put(parameterIndex, "'" + escapeSingleQuotes(x) + "'"); } } @Override - public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setBytes(int parameterIndex, byte[] x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.BINARY; + Binary binary = new Binary(x); + this.parameters.put(parameterIndex, binary.getStringValue(TSFileConfig.STRING_CHARSET)); } @Override - public void setBlob(int parameterIndex, Blob x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setDate(int parameterIndex, Date x) throws SQLException { + checkParameterIndex(parameterIndex); + DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); + String dateStr = dateFormat.format(x); + parameterValues[parameterIndex - 1] = dateStr; + parameterTypes[parameterIndex - 1] = Types.VARCHAR; + this.parameters.put(parameterIndex, "'" + dateStr + "'"); } @Override - public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { + setDate(parameterIndex, x); } @Override - public void setBlob(int parameterIndex, InputStream inputStream, long length) - throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setTime(int parameterIndex, Time x) throws SQLException { + checkParameterIndex(parameterIndex); + try { + long time = x.getTime(); + String timeprecision = client.getProperties().getTimestampPrecision(); + switch (timeprecision.toLowerCase()) { + case "ms": + break; + case "us": + time = time * 1000; + break; + case "ns": + time = time * 1000000; + break; + default: + break; + } + parameterValues[parameterIndex - 1] = time; + parameterTypes[parameterIndex - 1] = Types.BIGINT; + this.parameters.put(parameterIndex, Long.toString(time)); + } catch (TException e) { + throw new SQLException("Failed to get time precision: " + e.getMessage(), e); + } } @Override - public void setBoolean(int parameterIndex, boolean x) { - this.parameters.put(parameterIndex, Boolean.toString(x)); + public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { + setTime(parameterIndex, x); } @Override - public void setByte(int parameterIndex, byte x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { + checkParameterIndex(parameterIndex); + ZonedDateTime zonedDateTime = + ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()), super.zoneId); + String tsStr = zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + parameterValues[parameterIndex - 1] = tsStr; + parameterTypes[parameterIndex - 1] = Types.VARCHAR; + this.parameters.put(parameterIndex, tsStr); } @Override - public void setBytes(int parameterIndex, byte[] x) throws SQLException { - Binary binary = new Binary(x); - this.parameters.put(parameterIndex, binary.getStringValue(TSFileConfig.STRING_CHARSET)); + public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { + setTimestamp(parameterIndex, x); } @Override - public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setObject(int parameterIndex, Object x) throws SQLException { + if (x == null) { + setNull(parameterIndex, Types.NULL); + } else if (x instanceof String) { + setString(parameterIndex, (String) x); + } else if (x instanceof Integer) { + setInt(parameterIndex, (Integer) x); + } else if (x instanceof Long) { + setLong(parameterIndex, (Long) x); + } else if (x instanceof Float) { + setFloat(parameterIndex, (Float) x); + } else if (x instanceof Double) { + setDouble(parameterIndex, (Double) x); + } else if (x instanceof Boolean) { + setBoolean(parameterIndex, (Boolean) x); + } else if (x instanceof Timestamp) { + setTimestamp(parameterIndex, (Timestamp) x); + } else if (x instanceof Date) { + setDate(parameterIndex, (Date) x); + } else if (x instanceof Time) { + setTime(parameterIndex, (Time) x); + } else if (x instanceof byte[]) { + setBytes(parameterIndex, (byte[]) x); + } else { + throw new SQLException( + String.format( + "Can't infer the SQL type for an instance of %s. Use setObject() with explicit type.", + x.getClass().getName())); + } } @Override - public void setCharacterStream(int parameterIndex, Reader reader, int length) - throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { + setObject(parameterIndex, x); } @Override - public void setCharacterStream(int parameterIndex, Reader reader, long length) + public void setObject(int parameterIndex, Object parameterObj, int targetSqlType, int scale) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + setObject(parameterIndex, parameterObj); } + private void checkParameterIndex(int index) throws SQLException { + if (index < 1 || index > parameterCount) { + throw new SQLException( + "Parameter index out of range: " + index + " (expected 1-" + parameterCount + ")"); + } + } + + private String escapeSingleQuotes(String value) { + return value.replace("'", "''"); + } + + // ================== Unsupported Methods ================== + @Override - public void setClob(int parameterIndex, Clob x) throws SQLException { + public void setArray(int parameterIndex, Array x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setClob(int parameterIndex, Reader reader) throws SQLException { + public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { + public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setDate(int parameterIndex, Date x) throws SQLException { - DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); - this.parameters.put(parameterIndex, "'" + dateFormat.format(x) + "'"); + public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { + public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setDouble(int parameterIndex, double x) { - this.parameters.put(parameterIndex, Double.toString(x)); + public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setFloat(int parameterIndex, float x) { - this.parameters.put(parameterIndex, Float.toString(x)); + public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { + try { + byte[] bytes = ReadWriteIOUtils.readBytes(x, length); + setBytes(parameterIndex, bytes); + } catch (IOException e) { + throw new SQLException("Failed to read binary stream: " + e.getMessage(), e); + } } @Override - public void setInt(int parameterIndex, int x) { - this.parameters.put(parameterIndex, Integer.toString(x)); + public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setLong(int parameterIndex, long x) { - this.parameters.put(parameterIndex, Long.toString(x)); + public void setBlob(int parameterIndex, Blob x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { + public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNCharacterStream(int parameterIndex, Reader value, long length) + public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNClob(int parameterIndex, NClob value) throws SQLException { + public void setByte(int parameterIndex, byte x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNClob(int parameterIndex, Reader reader) throws SQLException { + public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { + public void setCharacterStream(int parameterIndex, Reader reader, int length) + throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNString(int parameterIndex, String value) throws SQLException { + public void setCharacterStream(int parameterIndex, Reader reader, long length) + throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNull(int parameterIndex, int sqlType) throws SQLException { - this.parameters.put(parameterIndex, "NULL"); + public void setClob(int parameterIndex, Clob x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException { - throw new SQLException(Constant.PARAMETER_NOT_NULL); + public void setClob(int parameterIndex, Reader reader) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setObject(int parameterIndex, Object x) throws SQLException { - if (x instanceof String) { - setString(parameterIndex, (String) x); - } else if (x instanceof Integer) { - setInt(parameterIndex, (Integer) x); - } else if (x instanceof Long) { - setLong(parameterIndex, (Long) x); - } else if (x instanceof Float) { - setFloat(parameterIndex, (Float) x); - } else if (x instanceof Double) { - setDouble(parameterIndex, (Double) x); - } else if (x instanceof Boolean) { - setBoolean(parameterIndex, (Boolean) x); - } else if (x instanceof Timestamp) { - setTimestamp(parameterIndex, (Timestamp) x); - } else if (x instanceof Date) { - setDate(parameterIndex, (Date) x); - } else if (x instanceof Blob) { - setBlob(parameterIndex, (Blob) x); - } else if (x instanceof Time) { - setTime(parameterIndex, (Time) x); - } else { - // Can't infer a type. - throw new SQLException( - String.format( - "Can''t infer the SQL type to use for an instance of %s. Use setObject() with" - + " an explicit Types value to specify the type to use.", - x.getClass().getName())); - } + public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { - if (!(x instanceof BigDecimal)) { - setObject(parameterIndex, x, targetSqlType, 0); - } else { - setObject(parameterIndex, x, targetSqlType, ((BigDecimal) x).scale()); - } + public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } - @SuppressWarnings({ - "squid:S3776", - "squid:S6541" - }) // ignore Cognitive Complexity of methods should not be too high - // ignore Methods should not perform too many tasks (aka Brain method) @Override - public void setObject(int parameterIndex, Object parameterObj, int targetSqlType, int scale) + public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException { - if (parameterObj == null) { - setNull(parameterIndex, java.sql.Types.OTHER); - } else { - try { - switch (targetSqlType) { - case Types.BOOLEAN: - if (parameterObj instanceof Boolean) { - setBoolean(parameterIndex, ((Boolean) parameterObj).booleanValue()); - break; - } else if (parameterObj instanceof String) { - if ("true".equalsIgnoreCase((String) parameterObj) - || "Y".equalsIgnoreCase((String) parameterObj)) { - setBoolean(parameterIndex, true); - } else if ("false".equalsIgnoreCase((String) parameterObj) - || "N".equalsIgnoreCase((String) parameterObj)) { - setBoolean(parameterIndex, false); - } else { - throw new SQLException( - "No conversion from " + parameterObj + " to Types.BOOLEAN possible."); - } - break; - } else if (parameterObj instanceof Number) { - int intValue = ((Number) parameterObj).intValue(); - - setBoolean(parameterIndex, intValue != 0); - - break; - } else { - throw new SQLException( - "No conversion from " + parameterObj + " to Types.BOOLEAN possible."); - } - - case Types.BIT: - case Types.TINYINT: - case Types.SMALLINT: - case Types.INTEGER: - case Types.BIGINT: - case Types.REAL: - case Types.FLOAT: - case Types.DOUBLE: - case Types.DECIMAL: - case Types.NUMERIC: - setNumericObject(parameterIndex, parameterObj, targetSqlType, scale); - break; - case Types.CHAR: - case Types.VARCHAR: - case Types.LONGVARCHAR: - if (parameterObj instanceof BigDecimal) { - setString( - parameterIndex, - StringUtils.fixDecimalExponent( - StringUtils.consistentToString((BigDecimal) parameterObj))); - } else { - setString(parameterIndex, parameterObj.toString()); - } - - break; - - case Types.CLOB: - if (parameterObj instanceof java.sql.Clob) { - setClob(parameterIndex, (java.sql.Clob) parameterObj); - } else { - setString(parameterIndex, parameterObj.toString()); - } - - break; - - case Types.BINARY: - case Types.VARBINARY: - case Types.LONGVARBINARY: - case Types.BLOB: - throw new SQLException(Constant.PARAMETER_SUPPORTED); - case Types.DATE: - case Types.TIMESTAMP: - java.util.Date parameterAsDate; - - if (parameterObj instanceof String) { - ParsePosition pp = new ParsePosition(0); - DateFormat sdf = - new SimpleDateFormat(getDateTimePattern((String) parameterObj, false), Locale.US); - parameterAsDate = sdf.parse((String) parameterObj, pp); - } else { - parameterAsDate = (Date) parameterObj; - } - - switch (targetSqlType) { - case Types.DATE: - if (parameterAsDate instanceof java.sql.Date) { - setDate(parameterIndex, (java.sql.Date) parameterAsDate); - } else { - setDate(parameterIndex, new java.sql.Date(parameterAsDate.getTime())); - } - - break; - - case Types.TIMESTAMP: - if (parameterAsDate instanceof java.sql.Timestamp) { - setTimestamp(parameterIndex, (java.sql.Timestamp) parameterAsDate); - } else { - setTimestamp(parameterIndex, new java.sql.Timestamp(parameterAsDate.getTime())); - } - - break; - default: - logger.error("No type was matched"); - break; - } - - break; - - case Types.TIME: - if (parameterObj instanceof String) { - DateFormat sdf = - new SimpleDateFormat(getDateTimePattern((String) parameterObj, true), Locale.US); - setTime(parameterIndex, new Time(sdf.parse((String) parameterObj).getTime())); - } else if (parameterObj instanceof Timestamp) { - Timestamp xT = (Timestamp) parameterObj; - setTime(parameterIndex, new Time(xT.getTime())); - } else { - setTime(parameterIndex, (Time) parameterObj); - } - - break; - - case Types.OTHER: - throw new SQLException(Constant.PARAMETER_SUPPORTED); // - default: - throw new SQLException(Constant.PARAMETER_SUPPORTED); // - } - } catch (SQLException ex) { - throw ex; - } catch (Exception ex) { - throw new SQLException(Constant.PARAMETER_SUPPORTED); // - } - } - } - - @SuppressWarnings({ - "squid:S3776", - "squid:S6541" - }) // ignore Cognitive Complexity of methods should not be too high - // ignore Methods should not perform too many tasks (aka Brain method) - private final String getDateTimePattern(String dt, boolean toTime) throws Exception { - // - // Special case - // - int dtLength = (dt != null) ? dt.length() : 0; - - if ((dtLength >= 8) && (dtLength <= 10)) { - int dashCount = 0; - boolean isDateOnly = true; - - for (int i = 0; i < dtLength; i++) { - char c = dt.charAt(i); - - if (!Character.isDigit(c) && (c != '-')) { - isDateOnly = false; - - break; - } - - if (c == '-') { - dashCount++; - } - } - - if (isDateOnly && (dashCount == 2)) { - return "yyyy-MM-dd"; - } - } - boolean colonsOnly = true; - - for (int i = 0; i < dtLength; i++) { - char c = dt.charAt(i); - - if (!Character.isDigit(c) && (c != ':')) { - colonsOnly = false; - - break; - } - } - - if (colonsOnly) { - return "HH:mm:ss"; - } - - int n; - int z; - int count; - int maxvecs; - char c; - char separator; - StringReader reader = new StringReader(dt + " "); - ArrayList vec = new ArrayList<>(); - ArrayList vecRemovelist = new ArrayList<>(); - Object[] nv = new Object[3]; - Object[] v; - nv[0] = Character.valueOf('y'); - nv[1] = new StringBuilder(); - nv[2] = Integer.valueOf(0); - vec.add(nv); - - if (toTime) { - nv = new Object[3]; - nv[0] = Character.valueOf('h'); - nv[1] = new StringBuilder(); - nv[2] = Integer.valueOf(0); - vec.add(nv); - } - - while ((z = reader.read()) != -1) { - separator = (char) z; - maxvecs = vec.size(); - - for (count = 0; count < maxvecs; count++) { - v = vec.get(count); - n = ((Integer) v[2]).intValue(); - c = getSuccessor(((Character) v[0]).charValue(), n); - - if (!Character.isLetterOrDigit(separator)) { - if ((c == ((Character) v[0]).charValue()) && (c != 'S')) { - vecRemovelist.add(v); - } else { - ((StringBuilder) v[1]).append(separator); - - if ((c == 'X') || (c == 'Y')) { - v[2] = Integer.valueOf(4); - } - } - } else { - if (c == 'X') { - c = 'y'; - nv = new Object[3]; - nv[1] = (new StringBuilder(((StringBuilder) v[1]).toString())).append('M'); - nv[0] = Character.valueOf('M'); - nv[2] = Integer.valueOf(1); - vec.add(nv); - } else if (c == 'Y') { - c = 'M'; - nv = new Object[3]; - nv[1] = (new StringBuilder(((StringBuilder) v[1]).toString())).append('d'); - nv[0] = Character.valueOf('d'); - nv[2] = Integer.valueOf(1); - vec.add(nv); - } - - ((StringBuilder) v[1]).append(c); - - if (c == ((Character) v[0]).charValue()) { - v[2] = Integer.valueOf(n + 1); - } else { - v[0] = Character.valueOf(c); - v[2] = Integer.valueOf(1); - } - } - } - - int size = vecRemovelist.size(); - - for (int i = 0; i < size; i++) { - v = vecRemovelist.get(i); - vec.remove(v); - } - - vecRemovelist.clear(); - } - - int size = vec.size(); - - for (int i = 0; i < size; i++) { - v = vec.get(i); - c = ((Character) v[0]).charValue(); - n = ((Integer) v[2]).intValue(); - - boolean bk = getSuccessor(c, n) != c; - boolean atEnd = (((c == 's') || (c == 'm') || ((c == 'h') && toTime)) && bk); - boolean finishesAtDate = (bk && (c == 'd') && !toTime); - boolean containsEnd = (((StringBuilder) v[1]).toString().indexOf('W') != -1); - - if ((!atEnd && !finishesAtDate) || (containsEnd)) { - vecRemovelist.add(v); - } - } - - size = vecRemovelist.size(); - - for (int i = 0; i < size; i++) { - vec.remove(vecRemovelist.get(i)); - } - - vecRemovelist.clear(); - v = vec.get(0); // might throw exception - - StringBuilder format = (StringBuilder) v[1]; - format.setLength(format.length() - 1); - - return format.toString(); - } - - @SuppressWarnings({"squid:S3776", "squid:S3358"}) // ignore Ternary operators should not be nested - // ignore Cognitive Complexity of methods should not be too high - private final char getSuccessor(char c, int n) { - return ((c == 'y') && (n == 2)) - ? 'X' - : (((c == 'y') && (n < 4)) - ? 'y' - : ((c == 'y') - ? 'M' - : (((c == 'M') && (n == 2)) - ? 'Y' - : (((c == 'M') && (n < 3)) - ? 'M' - : ((c == 'M') - ? 'd' - : (((c == 'd') && (n < 2)) - ? 'd' - : ((c == 'd') - ? 'H' - : (((c == 'H') && (n < 2)) - ? 'H' - : ((c == 'H') - ? 'm' - : (((c == 'm') && (n < 2)) - ? 'm' - : ((c == 'm') - ? 's' - : (((c == 's') && (n < 2)) - ? 's' - : 'W')))))))))))); - } - - @SuppressWarnings({ - "squid:S3776", - "squid:S6541" - }) // ignore Cognitive Complexity of methods should not be too high - // ignore Methods should not perform too many tasks (aka Brain method) - private void setNumericObject( - int parameterIndex, Object parameterObj, int targetSqlType, int scale) throws SQLException { - Number parameterAsNum; - - if (parameterObj instanceof Boolean) { - parameterAsNum = - ((Boolean) parameterObj).booleanValue() ? Integer.valueOf(1) : Integer.valueOf(0); - } else if (parameterObj instanceof String) { - switch (targetSqlType) { - case Types.BIT: - if ("1".equals(parameterObj) || "0".equals(parameterObj)) { - parameterAsNum = Integer.valueOf((String) parameterObj); - } else { - boolean parameterAsBoolean = "true".equalsIgnoreCase((String) parameterObj); - - parameterAsNum = parameterAsBoolean ? Integer.valueOf(1) : Integer.valueOf(0); - } - - break; - - case Types.TINYINT: - case Types.SMALLINT: - case Types.INTEGER: - parameterAsNum = Integer.valueOf((String) parameterObj); - - break; - - case Types.BIGINT: - parameterAsNum = Long.valueOf((String) parameterObj); - - break; - - case Types.REAL: - parameterAsNum = Float.valueOf((String) parameterObj); - - break; - - case Types.FLOAT: - case Types.DOUBLE: - parameterAsNum = Double.valueOf((String) parameterObj); - - break; - - case Types.DECIMAL: - case Types.NUMERIC: - default: - parameterAsNum = new java.math.BigDecimal((String) parameterObj); - } - } else { - parameterAsNum = (Number) parameterObj; - } - - switch (targetSqlType) { - case Types.BIT: - case Types.TINYINT: - case Types.SMALLINT: - case Types.INTEGER: - setInt(parameterIndex, parameterAsNum.intValue()); - break; - - case Types.BIGINT: - setLong(parameterIndex, parameterAsNum.longValue()); - break; - - case Types.REAL: - setFloat(parameterIndex, parameterAsNum.floatValue()); - break; - - case Types.FLOAT: - setFloat(parameterIndex, parameterAsNum.floatValue()); - break; - case Types.DOUBLE: - setDouble(parameterIndex, parameterAsNum.doubleValue()); - - break; - - case Types.DECIMAL: - case Types.NUMERIC: - if (parameterAsNum instanceof java.math.BigDecimal) { - BigDecimal scaledBigDecimal = null; - - try { - scaledBigDecimal = ((java.math.BigDecimal) parameterAsNum).setScale(scale); - } catch (ArithmeticException ex) { - try { - scaledBigDecimal = - ((java.math.BigDecimal) parameterAsNum).setScale(scale, BigDecimal.ROUND_HALF_UP); - } catch (ArithmeticException arEx) { - throw new SQLException( - "Can't set scale of '" - + scale - + "' for DECIMAL argument '" - + parameterAsNum - + "'"); - } - } - - setBigDecimal(parameterIndex, scaledBigDecimal); - } else if (parameterAsNum instanceof java.math.BigInteger) { - setBigDecimal( - parameterIndex, - new java.math.BigDecimal((java.math.BigInteger) parameterAsNum, scale)); - } else { - setBigDecimal(parameterIndex, BigDecimal.valueOf(parameterAsNum.doubleValue())); - } - - break; - default: - } - } - - @Override - public void setRef(int parameterIndex, Ref x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setRowId(int parameterIndex, RowId x) throws SQLException { - throw new SQLException(METHOD_NOT_SUPPORTED_STRING); + public void setNClob(int parameterIndex, NClob value) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { - throw new SQLException(METHOD_NOT_SUPPORTED_STRING); + public void setNClob(int parameterIndex, Reader reader) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setShort(int parameterIndex, short x) throws SQLException { + public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setString(int parameterIndex, String x) { - if (x == null) { - this.parameters.put(parameterIndex, null); - } else { - this.parameters.put(parameterIndex, "'" + escapeSingleQuotes(x) + "'"); - } - } - - private String escapeSingleQuotes(String value) { - // Escape single quotes with double single quotes - return value.replace("'", "''"); + public void setNString(int parameterIndex, String value) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setTime(int parameterIndex, Time x) throws SQLException { - try { - long time = x.getTime(); - String timeprecision = client.getProperties().getTimestampPrecision(); - switch (timeprecision.toLowerCase()) { - case "ms": - break; - case "us": - time = time * 1000; - break; - case "ns": - time = time * 1000000; - break; - default: - break; - } - setLong(parameterIndex, time); - } catch (TException e) { - logger.error( - String.format("set time error when iotdb prepared statement :%s ", e.getMessage())); - } + public void setRef(int parameterIndex, Ref x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { - try { - ZonedDateTime zonedDateTime = null; - long time = x.getTime(); - String timeprecision = client.getProperties().getTimestampPrecision(); - switch (timeprecision.toLowerCase()) { - case "ms": - break; - case "us": - time = time * 1000; - break; - case "ns": - time = time * 1000000; - break; - default: - break; - } - if (cal != null) { - zonedDateTime = - ZonedDateTime.ofInstant( - Instant.ofEpochMilli(time), ZoneId.of(cal.getTimeZone().getID())); - } else { - zonedDateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(time), super.zoneId); - } - this.parameters.put( - parameterIndex, zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); - } catch (TException e) { - logger.error( - String.format("set time error when iotdb prepared statement :%s ", e.getMessage())); - } + public void setRowId(int parameterIndex, RowId x) throws SQLException { + throw new SQLException(METHOD_NOT_SUPPORTED_STRING); } @Override - public void setTimestamp(int parameterIndex, Timestamp x) { - ZonedDateTime zonedDateTime = - ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()), super.zoneId); - this.parameters.put( - parameterIndex, zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { + throw new SQLException(METHOD_NOT_SUPPORTED_STRING); } @Override - public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { - ZonedDateTime zonedDateTime = null; - if (cal != null) { - zonedDateTime = - ZonedDateTime.ofInstant( - Instant.ofEpochMilli(x.getTime()), ZoneId.of(cal.getTimeZone().getID())); - } else { - zonedDateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()), super.zoneId); - } - this.parameters.put( - parameterIndex, zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + public void setShort(int parameterIndex, short x) throws SQLException { + setInt(parameterIndex, x); } @Override @@ -1010,16 +753,14 @@ public void setUnicodeStream(int parameterIndex, InputStream x, int length) thro throw new SQLException(Constant.PARAMETER_SUPPORTED); } + // ================== Helper Methods for Backward Compatibility ================== + private String createCompleteSql(final String sql, Map parameters) throws SQLException { List parts = splitSqlStatement(sql); StringBuilder newSql = new StringBuilder(parts.get(0)); for (int i = 1; i < parts.size(); i++) { - if (logger.isDebugEnabled()) { - logger.debug("SQL {}", sql); - logger.debug("parameters {}", parameters.size()); - } if (!parameters.containsKey(i)) { throw new SQLException("Parameter #" + i + " is unset"); } @@ -1043,15 +784,12 @@ private List splitSqlStatement(final String sql) { } switch (c) { case '\'': - // skip something like 'xxxxx' apCount++; break; case '\\': - // skip something like \r\n skip = true; break; case '?': - // for input like: select a from 'bc' where d, 'bc' will be skipped if ((apCount & 1) == 0) { parts.add(sql.substring(off, i)); off = i + 1; diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java index 93a922070db2..8cb0a32417f2 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java @@ -55,7 +55,7 @@ public class IoTDBStatement implements Statement { private final IoTDBConnection connection; - private ResultSet resultSet = null; + protected ResultSet resultSet = null; private int fetchSize; private int maxRows = 0; @@ -66,7 +66,7 @@ public class IoTDBStatement implements Statement { * Timeout of query can be set by users. Unit: s. A negative number means using the default * configuration of server. And value 0 will disable the function of query timeout. */ - private int queryTimeout = -1; + protected int queryTimeout = -1; protected IClientRPCService.Iface client; private List batchSQLList; @@ -82,7 +82,7 @@ public class IoTDBStatement implements Statement { /** Add SQLWarnings to the warningChain if needed. */ private SQLWarning warningChain = null; - private long sessionId; + protected long sessionId; private long stmtId = -1; private long queryId = -1; diff --git a/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java index 2ae65dfed2ae..aa932cda5f22 100644 --- a/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java +++ b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java @@ -22,23 +22,27 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.IClientRPCService.Iface; +import org.apache.iotdb.service.rpc.thrift.TSExecutePreparedReq; import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq; import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp; +import org.apache.iotdb.service.rpc.thrift.TSPrepareReq; +import org.apache.iotdb.service.rpc.thrift.TSPrepareResp; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.sql.SQLException; import java.sql.Timestamp; import java.sql.Types; import java.time.ZoneId; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -46,6 +50,7 @@ public class IoTDBPreparedStatementTest { @Mock TSExecuteStatementResp execStatementResp; + @Mock TSPrepareResp prepareResp; @Mock TSStatus getOperationStatusResp; private ZoneId zoneId = ZoneId.systemDefault(); @Mock private IoTDBConnection connection; @@ -62,6 +67,53 @@ public void before() throws Exception { when(execStatementResp.getQueryId()).thenReturn(queryId); when(client.executeStatementV2(any(TSExecuteStatementReq.class))).thenReturn(execStatementResp); + + // Mock for prepareStatement - dynamically calculate parameter count from SQL + when(client.prepareStatement(any(TSPrepareReq.class))) + .thenAnswer( + new Answer() { + @Override + public TSPrepareResp answer(InvocationOnMock invocation) throws Throwable { + TSPrepareReq req = invocation.getArgument(0); + String sql = req.getSql(); + int paramCount = countQuestionMarks(sql); + + TSPrepareResp resp = new TSPrepareResp(); + resp.setStatus(Status_SUCCESS); + resp.setParameterCount(paramCount); + return resp; + } + }); + + // Mock for executePreparedStatement + when(client.executePreparedStatement(any(TSExecutePreparedReq.class))) + .thenReturn(execStatementResp); + } + + /** Count the number of '?' placeholders in a SQL string, ignoring those inside quotes */ + private int countQuestionMarks(String sql) { + int count = 0; + boolean inSingleQuote = false; + boolean inDoubleQuote = false; + + for (int i = 0; i < sql.length(); i++) { + char c = sql.charAt(i); + + if (c == '\'' && !inDoubleQuote) { + // Check for escaped quote + if (i + 1 < sql.length() && sql.charAt(i + 1) == '\'') { + i++; // Skip escaped quote + } else { + inSingleQuote = !inSingleQuote; + } + } else if (c == '"' && !inSingleQuote) { + inDoubleQuote = !inDoubleQuote; + } else if (c == '?' && !inSingleQuote && !inDoubleQuote) { + count++; + } + } + + return count; } @SuppressWarnings("resource") @@ -73,23 +125,27 @@ public void testNonParameterized() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + // Verify executePreparedStatement was called (new behavior) + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // Non-parameterized query should have empty parameters + assertTrue( + argument.getValue().getParameters() == null + || argument.getValue().getParameters().isEmpty()); } @SuppressWarnings("resource") @Test public void unusedArgument() throws SQLException { + // SQL with no parameters - setting a parameter should throw an exception String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-1 0:13:00"; IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "123"); - assertFalse(ps.execute()); + // In the new server-side prepared statement implementation, setting a parameter + // that doesn't exist in the SQL throws an exception + assertThrows(SQLException.class, () -> ps.setString(1, "123")); } @SuppressWarnings("resource") @@ -111,12 +167,11 @@ public void oneIntArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setInt(1, 123); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123 and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // Verify parameters were sent + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -128,12 +183,10 @@ public void oneLongArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setLong(1, 123); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123 and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -145,12 +198,10 @@ public void oneFloatArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setFloat(1, 123.133f); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123.133 and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -162,12 +213,10 @@ public void oneDoubleArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setDouble(1, 123.456); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123.456 and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -179,12 +228,10 @@ public void oneBooleanArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setBoolean(1, false); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < false and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -196,12 +243,10 @@ public void oneStringArgument1() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setString(1, "'abcde'"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < '''abcde''' and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -213,12 +258,10 @@ public void oneStringArgument2() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setString(1, "\"abcde\""); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < '\"abcde\"' and time > 2017-11-1 0:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -229,11 +272,10 @@ public void oneStringArgument3() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setString(1, "temperature"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, 'temperature' FROM root.ln.wf01.wt01", argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -244,12 +286,10 @@ public void oneTimeLongArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setLong(1, 1233); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE time > 1233", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -260,12 +300,10 @@ public void oneTimeTimestampArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setTimestamp(1, Timestamp.valueOf("2017-11-01 00:13:00")); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE time > 2017-11-01T00:13:00", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -278,12 +316,10 @@ public void escapingOfStringArgument() throws Exception { ps.setLong(1, 1333); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE status = '134' and temperature = 1333", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -297,12 +333,10 @@ public void pastingIntoEscapedQuery() throws Exception { ps.setDouble(1, -1323.0); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE status = '\\044e' || temperature = -1323.0", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -321,12 +355,10 @@ public void testInsertStatement1() throws Exception { ps.setString(7, "'abc'"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f) VALUES(12324,false,123,123234345,123.423,-1323.0,'''abc''')", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -347,12 +379,10 @@ public void testInsertStatement2() throws Exception { ps.setString(9, "'abc'"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f,g,h) VALUES(2017-11-01T00:13:00,false,123,123234345,123.423,-1323.0,'\"abc\"','abc','''abc''')", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @Test @@ -370,12 +400,10 @@ public void testInsertStatement3() throws Exception { ps.setObject(7, "\"abc\"", Types.VARCHAR); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "INSERT INTO root.ln.wf01.wt02(time,a,b,c,d,e,f) VALUES(2020-01-01T10:10:10,false,123,123234345,123.423,-1323.0,'\"abc\"')", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @Test @@ -393,12 +421,10 @@ public void testInsertStatement4() throws Exception { ps.setObject(7, "abc", Types.VARCHAR); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "INSERT INTO root.ln.wf01.wt02(time,a,b,c,d,e,f) VALUES(2020-01-01T10:10:10,false,123,123234345,123.423,-1323.0,'abc')", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } // ========== Table Model SQL Injection Prevention Tests ========== @@ -415,12 +441,11 @@ public void testTableModelLoginInjectionWithComment() throws Exception { ps.setString(2, "password"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT * FROM users WHERE username = 'admin'' --' AND password = 'password'", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // SQL injection is prevented by using prepared statements with parameterized queries + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -435,12 +460,11 @@ public void testTableModelLoginInjectionWithORCondition() throws Exception { ps.setString(2, "' OR '1'='1"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT * FROM users WHERE username = 'admin' AND password = ''' OR ''1''=''1'", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // SQL injection is prevented by using prepared statements with parameterized queries + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -453,12 +477,11 @@ public void testTableModelQueryWithMultipleInjectionVectors() throws Exception { ps.setString(1, "'; DROP TABLE users;"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT * FROM users WHERE email = '''; DROP TABLE users;'", - argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // SQL injection is prevented by using prepared statements with parameterized queries + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -471,10 +494,10 @@ public void testTableModelString1() throws Exception { ps.setString(1, "a'b"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals("SELECT * FROM users WHERE password = 'a''b'", argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -487,10 +510,10 @@ public void testTableModelString2() throws Exception { ps.setString(1, "a\'b"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals("SELECT * FROM users WHERE password = 'a''b'", argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -503,11 +526,10 @@ public void testTableModelString3() throws Exception { ps.setString(1, "a\\'b"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT * FROM users WHERE password = 'a\\''b'", argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -520,11 +542,10 @@ public void testTableModelString4() throws Exception { ps.setString(1, "a\\\'b"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals( - "SELECT * FROM users WHERE password = 'a\\''b'", argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } @SuppressWarnings("resource") @@ -537,9 +558,9 @@ public void testTableModelStringWithNull() throws Exception { ps.setString(1, null); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecuteStatementReq.class); - verify(client).executeStatementV2(argument.capture()); - assertEquals("SELECT * FROM users WHERE email = null", argument.getValue().getStatement()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index 167a1fa914fd..b8248801f741 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -53,6 +53,7 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.protocol.session.IClientSession; +import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.protocol.thrift.OperationType; import org.apache.iotdb.db.queryengine.common.SessionInfo; @@ -75,6 +76,7 @@ import org.apache.iotdb.db.queryengine.plan.analyze.schema.ISchemaFetcher; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; import org.apache.iotdb.db.queryengine.plan.execution.IQueryExecution; +import org.apache.iotdb.db.queryengine.plan.execution.config.session.PreparedStatementMemoryManager; import org.apache.iotdb.db.queryengine.plan.parser.ASTVisitor; import org.apache.iotdb.db.queryengine.plan.parser.StatementGenerator; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; @@ -89,7 +91,15 @@ import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.cache.TableId; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.cache.TreeDeviceSchemaCacheManager; import org.apache.iotdb.db.queryengine.plan.relational.security.TreeAccessCheckContext; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ParameterExtractor; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BinaryLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NullLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SetSqlDialect; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StringLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Use; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.ParsingException; import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; @@ -146,9 +156,11 @@ import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq; import org.apache.iotdb.service.rpc.thrift.TSCreateSchemaTemplateReq; import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq; +import org.apache.iotdb.service.rpc.thrift.TSDeallocatePreparedReq; import org.apache.iotdb.service.rpc.thrift.TSDeleteDataReq; import org.apache.iotdb.service.rpc.thrift.TSDropSchemaTemplateReq; import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementReq; +import org.apache.iotdb.service.rpc.thrift.TSExecutePreparedReq; import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq; import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp; import org.apache.iotdb.service.rpc.thrift.TSFastLastDataQueryForOneDeviceReq; @@ -169,6 +181,8 @@ import org.apache.iotdb.service.rpc.thrift.TSLastDataQueryReq; import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq; import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp; +import org.apache.iotdb.service.rpc.thrift.TSPrepareReq; +import org.apache.iotdb.service.rpc.thrift.TSPrepareResp; import org.apache.iotdb.service.rpc.thrift.TSProtocolVersion; import org.apache.iotdb.service.rpc.thrift.TSPruneSchemaTemplateReq; import org.apache.iotdb.service.rpc.thrift.TSQueryDataSet; @@ -1488,6 +1502,225 @@ public TSStatus closeOperation(TSCloseOperationReq req) { COORDINATOR::cleanupQueryExecution); } + // ========================= PreparedStatement RPC Methods ========================= + + @Override + public TSPrepareResp prepareStatement(TSPrepareReq req) { + IClientSession clientSession = SESSION_MANAGER.getCurrSessionAndUpdateIdleTime(); + if (!SESSION_MANAGER.checkLogin(clientSession)) { + return new TSPrepareResp(getNotLoggedInStatus()); + } + + try { + String sql = req.getSql(); + String statementName = req.getStatementName(); + + if (clientSession.getPreparedStatement(statementName) != null) { + return new TSPrepareResp( + RpcUtils.getStatus( + TSStatusCode.EXECUTE_STATEMENT_ERROR, + String.format("Prepared statement '%s' already exists", statementName))); + } + + org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement = + relationSqlParser.createStatement(sql, clientSession.getZoneId(), clientSession); + + if (statement == null) { + return new TSPrepareResp( + RpcUtils.getStatus(TSStatusCode.SQL_PARSE_ERROR, "Failed to parse SQL: " + sql)); + } + + int parameterCount = ParameterExtractor.getParameterCount(statement); + + long memorySizeInBytes = statement.ramBytesUsed(); + + PreparedStatementMemoryManager.getInstance().allocate(statementName, memorySizeInBytes); + + PreparedStatementInfo info = + new PreparedStatementInfo(statementName, statement, memorySizeInBytes); + clientSession.addPreparedStatement(statementName, info); + + TSPrepareResp resp = new TSPrepareResp(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS)); + resp.setParameterCount(parameterCount); + return resp; + } catch (Exception e) { + return new TSPrepareResp( + onQueryException( + e, OperationType.EXECUTE_STATEMENT.getName(), TSStatusCode.INTERNAL_SERVER_ERROR)); + } + } + + @Override + public TSExecuteStatementResp executePreparedStatement(TSExecutePreparedReq req) { + boolean finished = false; + long queryId = Long.MIN_VALUE; + IClientSession clientSession = SESSION_MANAGER.getCurrSessionAndUpdateIdleTime(); + + if (!SESSION_MANAGER.checkLogin(clientSession)) { + return RpcUtils.getTSExecuteStatementResp(getNotLoggedInStatus()); + } + + long startTime = System.nanoTime(); + Throwable t = null; + try { + String statementName = req.getStatementName(); + + PreparedStatementInfo preparedInfo = clientSession.getPreparedStatement(statementName); + if (preparedInfo == null) { + return RpcUtils.getTSExecuteStatementResp( + RpcUtils.getStatus( + TSStatusCode.EXECUTE_STATEMENT_ERROR, + String.format("Prepared statement '%s' does not exist", statementName))); + } + + List parameters = deserializeParameters(req.getParameters()); + + org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement = + preparedInfo.getSql(); + + int expectedCount = ParameterExtractor.getParameterCount(statement); + if (parameters.size() != expectedCount) { + return RpcUtils.getTSExecuteStatementResp( + RpcUtils.getStatus( + TSStatusCode.EXECUTE_STATEMENT_ERROR, + String.format( + "Parameter count mismatch: expected %d, got %d", + expectedCount, parameters.size()))); + } + + // Request query ID + queryId = SESSION_MANAGER.requestQueryId(clientSession, null); + + // Execute using Coordinator with external parameters + long timeout = req.isSetTimeout() ? req.getTimeout() : config.getQueryTimeoutThreshold(); + ExecutionResult result = + COORDINATOR.executeForTableModel( + statement, + relationSqlParser, + clientSession, + queryId, + SESSION_MANAGER.getSessionInfo(clientSession), + "EXECUTE " + statementName, + metadata, + timeout, + true, + parameters); + + if (result.status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && result.status.code != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + finished = true; + return RpcUtils.getTSExecuteStatementResp(result.status); + } + + IQueryExecution queryExecution = COORDINATOR.getQueryExecution(queryId); + + try (SetThreadName threadName = new SetThreadName(result.queryId.getId())) { + TSExecuteStatementResp resp; + if (queryExecution != null && queryExecution.isQuery()) { + resp = createResponse(queryExecution.getDatasetHeader(), queryId); + resp.setStatus(result.status); + int fetchSize = + req.isSetFetchSize() ? req.getFetchSize() : config.getThriftMaxFrameSize(); + finished = setResultForPrepared.apply(resp, queryExecution, fetchSize); + resp.setMoreData(!finished); + } else { + finished = true; + resp = RpcUtils.getTSExecuteStatementResp(result.status); + } + return resp; + } + } catch (Exception e) { + finished = true; + t = e; + return RpcUtils.getTSExecuteStatementResp( + onQueryException( + e, OperationType.EXECUTE_STATEMENT.getName(), TSStatusCode.INTERNAL_SERVER_ERROR)); + } finally { + long currentOperationCost = System.nanoTime() - startTime; + if (finished) { + COORDINATOR.cleanupQueryExecution(queryId, null, t); + } + COORDINATOR.recordExecutionTime(queryId, currentOperationCost); + } + } + + @Override + public TSStatus deallocatePreparedStatement(TSDeallocatePreparedReq req) { + IClientSession clientSession = SESSION_MANAGER.getCurrSessionAndUpdateIdleTime(); + if (!SESSION_MANAGER.checkLogin(clientSession)) { + return getNotLoggedInStatus(); + } + + try { + String statementName = req.getStatementName(); + + PreparedStatementInfo removedInfo = clientSession.removePreparedStatement(statementName); + if (removedInfo == null) { + return RpcUtils.getStatus( + TSStatusCode.EXECUTE_STATEMENT_ERROR, + String.format("Prepared statement '%s' does not exist", statementName)); + } + + PreparedStatementMemoryManager.getInstance().release(removedInfo.getMemorySizeInBytes()); + + return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS); + } catch (Exception e) { + return onQueryException( + e, OperationType.EXECUTE_STATEMENT.getName(), TSStatusCode.INTERNAL_SERVER_ERROR); + } + } + + private List deserializeParameters(List params) { + List literals = new ArrayList<>(); + for (ByteBuffer buf : params) { + buf.rewind(); + byte type = buf.get(); + switch (type) { + case 0x00: // Null + literals.add(new NullLiteral()); + break; + case 0x01: // Boolean + boolean boolVal = buf.get() != 0; + literals.add(new BooleanLiteral(boolVal ? "true" : "false")); + break; + case 0x02: // Long + long longVal = buf.getLong(); + literals.add(new LongLiteral(String.valueOf(longVal))); + break; + case 0x03: // Double + double doubleVal = buf.getDouble(); + literals.add(new DoubleLiteral(doubleVal)); + break; + case 0x04: // String + int strLen = buf.getInt(); + byte[] strBytes = new byte[strLen]; + buf.get(strBytes); + literals.add( + new StringLiteral(new String(strBytes, java.nio.charset.StandardCharsets.UTF_8))); + break; + case 0x05: // Binary + int binLen = buf.getInt(); + byte[] binBytes = new byte[binLen]; + buf.get(binBytes); + literals.add(new BinaryLiteral(binBytes)); + break; + default: + throw new IllegalArgumentException("Unknown parameter type: " + type); + } + } + return literals; + } + + private final SelectResult setResultForPrepared = + (resp, queryExecution, fetchSize) -> { + Pair pair = + QueryDataSetUtils.convertTsBlockByFetchSize(queryExecution, fetchSize); + resp.setQueryDataSet(pair.left); + return pair.right; + }; + + // ========================= End PreparedStatement RPC Methods ========================= + @Override public TSGetTimeZoneResp getTimeZone(long sessionId) { try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index 3210d277d861..87ae0c65539d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -404,6 +404,47 @@ public ExecutionResult executeForTableModel( Metadata metadata, long timeOut, boolean userQuery) { + // Delegate to overloaded version with empty parameters + return executeForTableModel( + statement, + sqlParser, + clientSession, + queryId, + session, + sql, + metadata, + timeOut, + userQuery, + Collections.emptyList()); + } + + /** + * Execute a table model statement with optional pre-bound parameters. Used by JDBC + * PreparedStatement to execute cached AST with serialized parameters. + * + * @param statement The AST to execute + * @param sqlParser SQL parser instance + * @param clientSession Current client session + * @param queryId Query ID + * @param session Session info + * @param sql SQL string for logging + * @param metadata Metadata instance + * @param timeOut Query timeout + * @param userQuery Whether this is a user query + * @param externalParameters List of Literal parameters to bind (empty for normal execution) + * @return ExecutionResult containing execution status and query ID + */ + public ExecutionResult executeForTableModel( + org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement, + SqlParser sqlParser, + IClientSession clientSession, + long queryId, + SessionInfo session, + String sql, + Metadata metadata, + long timeOut, + boolean userQuery, + List externalParameters) { return execution( queryId, session, @@ -417,7 +458,8 @@ public ExecutionResult executeForTableModel( queryContext, metadata, timeOut > 0 ? timeOut : CONFIG.getQueryTimeoutThreshold(), - startTime))); + startTime, + externalParameters))); } public ExecutionResult executeForTableModel( @@ -481,7 +523,8 @@ private IQueryExecution createQueryExecutionForTableModel( final MPPQueryContext queryContext, final Metadata metadata, final long timeOut, - final long startTime) { + final long startTime, + final List externalParameters) { queryContext.setTimeOut(timeOut); queryContext.setStartTime(startTime); if (statement instanceof DropDB @@ -561,7 +604,11 @@ private IQueryExecution createQueryExecutionForTableModel( List parameters = Collections.emptyList(); Map, Expression> parameterLookup = Collections.emptyMap(); - if (statement instanceof Execute) { + // Handle external parameters from JDBC PreparedStatement (highest priority) + if (externalParameters != null && !externalParameters.isEmpty()) { + parameterLookup = ParameterExtractor.bindParameters(statement, externalParameters); + parameters = new ArrayList<>(externalParameters); + } else if (statement instanceof Execute) { Execute executeStatement = (Execute) statement; String statementName = executeStatement.getStatementName().getValue(); diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift index 48afb89d3366..e751b9b4e713 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift @@ -167,6 +167,36 @@ struct TSCloseOperationReq { 4: optional string preparedStatementName } +// PreparedStatement - PREPARE +// Parses SQL and caches AST in session for later execution +struct TSPrepareReq { + 1: required i64 sessionId + 2: required string sql // SQL with ? placeholders + 3: required string statementName // Name to identify this prepared statement +} + +struct TSPrepareResp { + 1: required common.TSStatus status + 2: optional i32 parameterCount // Number of ? placeholders in SQL +} + +// PreparedStatement - EXECUTE +// Executes a prepared statement with bound parameters +struct TSExecutePreparedReq { + 1: required i64 sessionId + 2: required string statementName // Name of the prepared statement + 3: required list parameters // Serialized parameter values + 4: optional i32 fetchSize + 5: optional i64 timeout +} + +// PreparedStatement - DEALLOCATE +// Releases a prepared statement and its resources +struct TSDeallocatePreparedReq { + 1: required i64 sessionId + 2: required string statementName // Name of the prepared statement to release +} + struct TSFetchResultsReq{ 1: required i64 sessionId 2: required string statement @@ -576,6 +606,13 @@ service IClientRPCService { common.TSStatus closeOperation(1:TSCloseOperationReq req); + // PreparedStatement operations + TSPrepareResp prepareStatement(1:TSPrepareReq req); + + TSExecuteStatementResp executePreparedStatement(1:TSExecutePreparedReq req); + + common.TSStatus deallocatePreparedStatement(1:TSDeallocatePreparedReq req); + TSGetTimeZoneResp getTimeZone(1:i64 sessionId); common.TSStatus setTimeZone(1:TSSetTimeZoneReq req); From 8c5c23805b7531c6141f6aca3d9439abf3b088e2 Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Sun, 18 Jan 2026 22:34:15 +0800 Subject: [PATCH 3/8] refactor: simplify PreparedStatement parameter serialization AND move serialization logic to shared service-rpc module --- .../apache/iotdb/jdbc/IoTDBConnection.java | 6 +- .../iotdb/jdbc/IoTDBPreparedStatement.java | 1118 ++++++++++------- .../jdbc/IoTDBTablePreparedStatement.java | 722 +++++++++++ .../jdbc/IoTDBPreparedStatementTest.java | 373 ++---- .../jdbc/IoTDBTablePreparedStatementTest.java | 240 ++++ .../rpc/stmt/PreparedParameterSerializer.java | 200 +++ .../thrift/impl/ClientRPCServiceImpl.java | 73 +- .../src/main/thrift/client.thrift | 7 +- 8 files changed, 1999 insertions(+), 740 deletions(-) create mode 100644 iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java create mode 100644 iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatementTest.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java index 51ebf7d727a0..54b148f25e4c 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java @@ -433,7 +433,11 @@ public CallableStatement prepareCall(String arg0, int arg1, int arg2, int arg3) @Override public PreparedStatement prepareStatement(String sql) throws SQLException { - return new IoTDBPreparedStatement(this, getClient(), sessionId, sql, zoneId, charset); + if (getSqlDialect().equals(Constant.TABLE_DIALECT)) { + return new IoTDBTablePreparedStatement(this, getClient(), sessionId, sql, zoneId, charset); + } else { + return new IoTDBPreparedStatement(this, getClient(), sessionId, sql, zoneId, charset); + } } @Override diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java index 032b9769df36..c92b6549bf9d 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBPreparedStatement.java @@ -19,16 +19,7 @@ package org.apache.iotdb.jdbc; -import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.rpc.RpcUtils; -import org.apache.iotdb.rpc.StatementExecutionException; -import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.IClientRPCService.Iface; -import org.apache.iotdb.service.rpc.thrift.TSDeallocatePreparedReq; -import org.apache.iotdb.service.rpc.thrift.TSExecutePreparedReq; -import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp; -import org.apache.iotdb.service.rpc.thrift.TSPrepareReq; -import org.apache.iotdb.service.rpc.thrift.TSPrepareResp; import org.apache.thrift.TException; import org.apache.tsfile.common.conf.TSFileConfig; @@ -40,11 +31,10 @@ import java.io.IOException; import java.io.InputStream; import java.io.Reader; +import java.io.StringReader; import java.math.BigDecimal; import java.net.URL; -import java.nio.ByteBuffer; import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; import java.sql.Array; import java.sql.Blob; import java.sql.Clob; @@ -62,6 +52,7 @@ import java.sql.Timestamp; import java.sql.Types; import java.text.DateFormat; +import java.text.ParsePosition; import java.text.SimpleDateFormat; import java.time.Instant; import java.time.ZoneId; @@ -71,31 +62,16 @@ import java.util.Calendar; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; -import java.util.UUID; public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedStatement { - private static final Logger logger = LoggerFactory.getLogger(IoTDBPreparedStatement.class); + private String sql; private static final String METHOD_NOT_SUPPORTED_STRING = "Method not supported"; + private static final Logger logger = LoggerFactory.getLogger(IoTDBPreparedStatement.class); - private final String sql; - private final String preparedStatementName; - private final int parameterCount; - - // Parameter values stored as objects for binary serialization - private final Object[] parameterValues; - private final int[] parameterTypes; - - // Parameter type constants for serialization - private static final byte TYPE_NULL = 0x00; - private static final byte TYPE_BOOLEAN = 0x01; - private static final byte TYPE_LONG = 0x02; - private static final byte TYPE_DOUBLE = 0x03; - private static final byte TYPE_STRING = 0x04; - private static final byte TYPE_BINARY = 0x05; - - /** save the SQL parameters as (paramLoc,paramValue) pairs for backward compatibility. */ + /** save the SQL parameters as (paramLoc,paramValue) pairs. */ private final Map parameters = new HashMap<>(); IoTDBPreparedStatement( @@ -108,42 +84,14 @@ public class IoTDBPreparedStatement extends IoTDBStatement implements PreparedSt throws SQLException { super(connection, client, sessionId, zoneId, charset); this.sql = sql; - this.preparedStatementName = generateStatementName(); - - // Send PREPARE request to server - TSPrepareReq prepareReq = new TSPrepareReq(); - prepareReq.setSessionId(sessionId); - prepareReq.setSql(sql); - prepareReq.setStatementName(preparedStatementName); - - try { - TSPrepareResp resp = client.prepareStatement(prepareReq); - RpcUtils.verifySuccess(resp.getStatus()); - - this.parameterCount = resp.isSetParameterCount() ? resp.getParameterCount() : 0; - this.parameterValues = new Object[parameterCount]; - this.parameterTypes = new int[parameterCount]; - - // Initialize all parameter types to NULL - for (int i = 0; i < parameterCount; i++) { - parameterTypes[i] = Types.NULL; - } - } catch (TException e) { - throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); - } catch (StatementExecutionException e) { - throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); - } } // Only for tests IoTDBPreparedStatement( IoTDBConnection connection, Iface client, Long sessionId, String sql, ZoneId zoneId) throws SQLException { - this(connection, client, sessionId, sql, zoneId, TSFileConfig.STRING_CHARSET); - } - - private String generateStatementName() { - return "jdbc_ps_" + UUID.randomUUID().toString().replace("-", ""); + super(connection, client, sessionId, zoneId, TSFileConfig.STRING_CHARSET); + this.sql = sql; } @Override @@ -154,186 +102,26 @@ public void addBatch() throws SQLException { @Override public void clearParameters() { this.parameters.clear(); - for (int i = 0; i < parameterCount; i++) { - parameterValues[i] = null; - parameterTypes[i] = Types.NULL; - } } @Override public boolean execute() throws SQLException { - TSExecuteStatementResp resp = executeInternal(); - return resp.isSetQueryDataSet() || resp.isSetQueryResult(); + return super.execute(createCompleteSql(sql, parameters)); } @Override public ResultSet executeQuery() throws SQLException { - TSExecuteStatementResp resp = executeInternal(); - return processQueryResult(resp); + return super.executeQuery(createCompleteSql(sql, parameters)); } @Override public int executeUpdate() throws SQLException { - executeInternal(); - return 0; // IoTDB doesn't return affected row count - } - - private TSExecuteStatementResp executeInternal() throws SQLException { - // Validate all parameters are set - for (int i = 0; i < parameterCount; i++) { - if (parameterTypes[i] == Types.NULL - && parameterValues[i] == null - && !parameters.containsKey(i + 1)) { - throw new SQLException("Parameter #" + (i + 1) + " is unset"); - } - } - - TSExecutePreparedReq req = new TSExecutePreparedReq(); - req.setSessionId(sessionId); - req.setStatementName(preparedStatementName); - req.setParameters(serializeParameters()); - - if (queryTimeout > 0) { - req.setTimeout(queryTimeout * 1000L); - } - - try { - TSExecuteStatementResp resp = client.executePreparedStatement(req); - RpcUtils.verifySuccess(resp.getStatus()); - return resp; - } catch (TException e) { - throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); - } catch (StatementExecutionException e) { - throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); - } - } - - private ResultSet processQueryResult(TSExecuteStatementResp resp) throws SQLException { - if (resp.isSetQueryDataSet() || resp.isSetQueryResult()) { - // Create ResultSet from response - this.resultSet = - new IoTDBJDBCResultSet( - this, - resp.getColumns(), - resp.getDataTypeList(), - resp.columnNameIndexMap, - resp.ignoreTimeStamp, - client, - sql, - resp.queryId, - sessionId, - resp.queryResult, - resp.tracingInfo, - (long) queryTimeout * 1000, - resp.isSetMoreData() && resp.isMoreData(), - zoneId); - return resultSet; - } - return null; - } - - /** - * Serialize parameters to binary format for transmission. Format: [type:1byte][value:variable] - */ - private List serializeParameters() { - List serialized = new ArrayList<>(); - for (int i = 0; i < parameterCount; i++) { - serialized.add(serializeParameter(i)); - } - return serialized; - } - - private ByteBuffer serializeParameter(int index) { - Object value = parameterValues[index]; - int type = parameterTypes[index]; - - if (value == null || type == Types.NULL) { - return ByteBuffer.wrap(new byte[] {TYPE_NULL}); - } - - switch (type) { - case Types.BOOLEAN: - ByteBuffer boolBuf = ByteBuffer.allocate(2); - boolBuf.put(TYPE_BOOLEAN); - boolBuf.put((byte) ((Boolean) value ? 1 : 0)); - boolBuf.flip(); - return boolBuf; - - case Types.INTEGER: - case Types.BIGINT: - ByteBuffer longBuf = ByteBuffer.allocate(9); - longBuf.put(TYPE_LONG); - longBuf.putLong(((Number) value).longValue()); - longBuf.flip(); - return longBuf; - - case Types.FLOAT: - case Types.DOUBLE: - ByteBuffer doubleBuf = ByteBuffer.allocate(9); - doubleBuf.put(TYPE_DOUBLE); - doubleBuf.putDouble(((Number) value).doubleValue()); - doubleBuf.flip(); - return doubleBuf; - - case Types.VARCHAR: - case Types.CHAR: - byte[] strBytes = ((String) value).getBytes(StandardCharsets.UTF_8); - ByteBuffer strBuf = ByteBuffer.allocate(5 + strBytes.length); - strBuf.put(TYPE_STRING); - strBuf.putInt(strBytes.length); - strBuf.put(strBytes); - strBuf.flip(); - return strBuf; - - case Types.BINARY: - case Types.VARBINARY: - byte[] binBytes = (byte[]) value; - ByteBuffer binBuf = ByteBuffer.allocate(5 + binBytes.length); - binBuf.put(TYPE_BINARY); - binBuf.putInt(binBytes.length); - binBuf.put(binBytes); - binBuf.flip(); - return binBuf; - - default: - // Fallback: serialize as string - String strValue = String.valueOf(value); - byte[] defaultBytes = strValue.getBytes(StandardCharsets.UTF_8); - ByteBuffer defaultBuf = ByteBuffer.allocate(5 + defaultBytes.length); - defaultBuf.put(TYPE_STRING); - defaultBuf.putInt(defaultBytes.length); - defaultBuf.put(defaultBytes); - defaultBuf.flip(); - return defaultBuf; - } - } - - @Override - public void close() throws SQLException { - if (!isClosed()) { - // Deallocate prepared statement on server - TSDeallocatePreparedReq req = new TSDeallocatePreparedReq(); - req.setSessionId(sessionId); - req.setStatementName(preparedStatementName); - - try { - TSStatus status = client.deallocatePreparedStatement(req); - if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - logger.warn("Failed to deallocate prepared statement: {}", status.getMessage()); - } - } catch (TException e) { - logger.warn("Error deallocating prepared statement", e); - } - } - super.close(); + return super.executeUpdate(createCompleteSql(sql, parameters)); } @Override public ResultSetMetaData getMetaData() throws SQLException { - if (resultSet != null) { - return resultSet.getMetaData(); - } - return null; + return getResultSet().getMetaData(); } @Override @@ -341,7 +129,7 @@ public ParameterMetaData getParameterMetaData() { return new ParameterMetaData() { @Override public int getParameterCount() { - return parameterCount; + return parameters.size(); } @Override @@ -351,26 +139,43 @@ public int isNullable(int param) { @Override public boolean isSigned(int param) { - int type = parameterTypes[param - 1]; - return type == Types.INTEGER - || type == Types.BIGINT - || type == Types.FLOAT - || type == Types.DOUBLE; + try { + return Integer.parseInt(parameters.get(param)) < 0; + } catch (Exception e) { + return false; + } } @Override public int getPrecision(int param) { - return 0; + return parameters.get(param).length(); } @Override public int getScale(int param) { - return 0; + try { + double d = Double.parseDouble(parameters.get(param)); + if (d >= 1) { // we only need the fraction digits + d = d - (long) d; + } + if (d == 0) { // nothing to count + return 0; + } + d *= 10; // shifts 1 digit to left + int count = 1; + while (d - (long) d != 0) { // keeps shifting until there are no more fractions + d *= 10; + count++; + } + return count; + } catch (Exception e) { + return 0; + } } @Override public int getParameterType(int param) { - return parameterTypes[param - 1]; + return 0; } @Override @@ -385,7 +190,7 @@ public String getParameterClassName(int param) { @Override public int getParameterMode(int param) { - return ParameterMetaData.parameterModeIn; + return 0; } @Override @@ -400,347 +205,799 @@ public boolean isWrapperFor(Class iface) { }; } - // ================== Parameter Setters ================== - @Override - public void setNull(int parameterIndex, int sqlType) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = null; - parameterTypes[parameterIndex - 1] = Types.NULL; - this.parameters.put(parameterIndex, "NULL"); - } - - @Override - public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException { - setNull(parameterIndex, sqlType); + public void setArray(int parameterIndex, Array x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setBoolean(int parameterIndex, boolean x) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = x; - parameterTypes[parameterIndex - 1] = Types.BOOLEAN; - this.parameters.put(parameterIndex, Boolean.toString(x)); + public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setInt(int parameterIndex, int x) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = (long) x; - parameterTypes[parameterIndex - 1] = Types.INTEGER; - this.parameters.put(parameterIndex, Integer.toString(x)); + public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setLong(int parameterIndex, long x) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = x; - parameterTypes[parameterIndex - 1] = Types.BIGINT; - this.parameters.put(parameterIndex, Long.toString(x)); + public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setFloat(int parameterIndex, float x) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = (double) x; - parameterTypes[parameterIndex - 1] = Types.FLOAT; - this.parameters.put(parameterIndex, Float.toString(x)); + public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setDouble(int parameterIndex, double x) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = x; - parameterTypes[parameterIndex - 1] = Types.DOUBLE; - this.parameters.put(parameterIndex, Double.toString(x)); + public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setString(int parameterIndex, String x) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = x; - parameterTypes[parameterIndex - 1] = Types.VARCHAR; - if (x == null) { - this.parameters.put(parameterIndex, null); - } else { - this.parameters.put(parameterIndex, "'" + escapeSingleQuotes(x) + "'"); + public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { + byte[] bytes = null; + try { + bytes = ReadWriteIOUtils.readBytes(x, length); + StringBuilder sb = new StringBuilder(); + for (byte b : bytes) { + sb.append(String.format("%02x", b)); + } + this.parameters.put(parameterIndex, "X'" + sb.toString() + "'"); + } catch (IOException e) { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } } @Override - public void setBytes(int parameterIndex, byte[] x) throws SQLException { - checkParameterIndex(parameterIndex); - parameterValues[parameterIndex - 1] = x; - parameterTypes[parameterIndex - 1] = Types.BINARY; - Binary binary = new Binary(x); - this.parameters.put(parameterIndex, binary.getStringValue(TSFileConfig.STRING_CHARSET)); + public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setDate(int parameterIndex, Date x) throws SQLException { - checkParameterIndex(parameterIndex); - DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); - String dateStr = dateFormat.format(x); - parameterValues[parameterIndex - 1] = dateStr; - parameterTypes[parameterIndex - 1] = Types.VARCHAR; - this.parameters.put(parameterIndex, "'" + dateStr + "'"); + public void setBlob(int parameterIndex, Blob x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { - setDate(parameterIndex, x); + public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setTime(int parameterIndex, Time x) throws SQLException { - checkParameterIndex(parameterIndex); - try { - long time = x.getTime(); - String timeprecision = client.getProperties().getTimestampPrecision(); - switch (timeprecision.toLowerCase()) { - case "ms": - break; - case "us": - time = time * 1000; - break; - case "ns": - time = time * 1000000; - break; - default: - break; - } - parameterValues[parameterIndex - 1] = time; - parameterTypes[parameterIndex - 1] = Types.BIGINT; - this.parameters.put(parameterIndex, Long.toString(time)); - } catch (TException e) { - throw new SQLException("Failed to get time precision: " + e.getMessage(), e); - } + public void setBlob(int parameterIndex, InputStream inputStream, long length) + throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { - setTime(parameterIndex, x); + public void setBoolean(int parameterIndex, boolean x) { + this.parameters.put(parameterIndex, Boolean.toString(x)); } @Override - public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { - checkParameterIndex(parameterIndex); - ZonedDateTime zonedDateTime = - ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()), super.zoneId); - String tsStr = zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); - parameterValues[parameterIndex - 1] = tsStr; - parameterTypes[parameterIndex - 1] = Types.VARCHAR; - this.parameters.put(parameterIndex, tsStr); + public void setByte(int parameterIndex, byte x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { - setTimestamp(parameterIndex, x); + public void setBytes(int parameterIndex, byte[] x) throws SQLException { + Binary binary = new Binary(x); + this.parameters.put(parameterIndex, binary.getStringValue(TSFileConfig.STRING_CHARSET)); } @Override - public void setObject(int parameterIndex, Object x) throws SQLException { - if (x == null) { - setNull(parameterIndex, Types.NULL); - } else if (x instanceof String) { - setString(parameterIndex, (String) x); - } else if (x instanceof Integer) { - setInt(parameterIndex, (Integer) x); - } else if (x instanceof Long) { - setLong(parameterIndex, (Long) x); - } else if (x instanceof Float) { - setFloat(parameterIndex, (Float) x); - } else if (x instanceof Double) { - setDouble(parameterIndex, (Double) x); - } else if (x instanceof Boolean) { - setBoolean(parameterIndex, (Boolean) x); - } else if (x instanceof Timestamp) { - setTimestamp(parameterIndex, (Timestamp) x); - } else if (x instanceof Date) { - setDate(parameterIndex, (Date) x); - } else if (x instanceof Time) { - setTime(parameterIndex, (Time) x); - } else if (x instanceof byte[]) { - setBytes(parameterIndex, (byte[]) x); - } else { - throw new SQLException( - String.format( - "Can't infer the SQL type for an instance of %s. Use setObject() with explicit type.", - x.getClass().getName())); - } + public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { - setObject(parameterIndex, x); + public void setCharacterStream(int parameterIndex, Reader reader, int length) + throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setObject(int parameterIndex, Object parameterObj, int targetSqlType, int scale) + public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException { - setObject(parameterIndex, parameterObj); - } - - private void checkParameterIndex(int index) throws SQLException { - if (index < 1 || index > parameterCount) { - throw new SQLException( - "Parameter index out of range: " + index + " (expected 1-" + parameterCount + ")"); - } - } - - private String escapeSingleQuotes(String value) { - return value.replace("'", "''"); + throw new SQLException(Constant.PARAMETER_SUPPORTED); } - // ================== Unsupported Methods ================== - @Override - public void setArray(int parameterIndex, Array x) throws SQLException { + public void setClob(int parameterIndex, Clob x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { + public void setClob(int parameterIndex, Reader reader) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { + public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setDate(int parameterIndex, Date x) throws SQLException { + DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); + this.parameters.put(parameterIndex, "'" + dateFormat.format(x) + "'"); } @Override - public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { + public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setDouble(int parameterIndex, double x) { + this.parameters.put(parameterIndex, Double.toString(x)); } @Override - public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { - try { - byte[] bytes = ReadWriteIOUtils.readBytes(x, length); - setBytes(parameterIndex, bytes); - } catch (IOException e) { - throw new SQLException("Failed to read binary stream: " + e.getMessage(), e); - } + public void setFloat(int parameterIndex, float x) { + this.parameters.put(parameterIndex, Float.toString(x)); } @Override - public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setInt(int parameterIndex, int x) { + this.parameters.put(parameterIndex, Integer.toString(x)); } @Override - public void setBlob(int parameterIndex, Blob x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setLong(int parameterIndex, long x) { + this.parameters.put(parameterIndex, Long.toString(x)); } @Override - public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { + public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setBlob(int parameterIndex, InputStream inputStream, long length) + public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setByte(int parameterIndex, byte x) throws SQLException { + public void setNClob(int parameterIndex, NClob value) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { + public void setNClob(int parameterIndex, Reader reader) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setCharacterStream(int parameterIndex, Reader reader, int length) - throws SQLException { + public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setCharacterStream(int parameterIndex, Reader reader, long length) - throws SQLException { + public void setNString(int parameterIndex, String value) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setClob(int parameterIndex, Clob x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setNull(int parameterIndex, int sqlType) throws SQLException { + this.parameters.put(parameterIndex, "NULL"); } @Override - public void setClob(int parameterIndex, Reader reader) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException { + throw new SQLException(Constant.PARAMETER_NOT_NULL); } @Override - public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setObject(int parameterIndex, Object x) throws SQLException { + if (x instanceof String) { + setString(parameterIndex, (String) x); + } else if (x instanceof Integer) { + setInt(parameterIndex, (Integer) x); + } else if (x instanceof Long) { + setLong(parameterIndex, (Long) x); + } else if (x instanceof Float) { + setFloat(parameterIndex, (Float) x); + } else if (x instanceof Double) { + setDouble(parameterIndex, (Double) x); + } else if (x instanceof Boolean) { + setBoolean(parameterIndex, (Boolean) x); + } else if (x instanceof Timestamp) { + setTimestamp(parameterIndex, (Timestamp) x); + } else if (x instanceof Date) { + setDate(parameterIndex, (Date) x); + } else if (x instanceof Blob) { + setBlob(parameterIndex, (Blob) x); + } else if (x instanceof Time) { + setTime(parameterIndex, (Time) x); + } else { + // Can't infer a type. + throw new SQLException( + String.format( + "Can''t infer the SQL type to use for an instance of %s. Use setObject() with" + + " an explicit Types value to specify the type to use.", + x.getClass().getName())); + } } @Override - public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { + if (!(x instanceof BigDecimal)) { + setObject(parameterIndex, x, targetSqlType, 0); + } else { + setObject(parameterIndex, x, targetSqlType, ((BigDecimal) x).scale()); + } } + @SuppressWarnings({ + "squid:S3776", + "squid:S6541" + }) // ignore Cognitive Complexity of methods should not be too high + // ignore Methods should not perform too many tasks (aka Brain method) @Override - public void setNCharacterStream(int parameterIndex, Reader value, long length) + public void setObject(int parameterIndex, Object parameterObj, int targetSqlType, int scale) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + if (parameterObj == null) { + setNull(parameterIndex, java.sql.Types.OTHER); + } else { + try { + switch (targetSqlType) { + case Types.BOOLEAN: + if (parameterObj instanceof Boolean) { + setBoolean(parameterIndex, ((Boolean) parameterObj).booleanValue()); + break; + } else if (parameterObj instanceof String) { + if ("true".equalsIgnoreCase((String) parameterObj) + || "Y".equalsIgnoreCase((String) parameterObj)) { + setBoolean(parameterIndex, true); + } else if ("false".equalsIgnoreCase((String) parameterObj) + || "N".equalsIgnoreCase((String) parameterObj)) { + setBoolean(parameterIndex, false); + } else { + throw new SQLException( + "No conversion from " + parameterObj + " to Types.BOOLEAN possible."); + } + break; + } else if (parameterObj instanceof Number) { + int intValue = ((Number) parameterObj).intValue(); + + setBoolean(parameterIndex, intValue != 0); + + break; + } else { + throw new SQLException( + "No conversion from " + parameterObj + " to Types.BOOLEAN possible."); + } + + case Types.BIT: + case Types.TINYINT: + case Types.SMALLINT: + case Types.INTEGER: + case Types.BIGINT: + case Types.REAL: + case Types.FLOAT: + case Types.DOUBLE: + case Types.DECIMAL: + case Types.NUMERIC: + setNumericObject(parameterIndex, parameterObj, targetSqlType, scale); + break; + case Types.CHAR: + case Types.VARCHAR: + case Types.LONGVARCHAR: + if (parameterObj instanceof BigDecimal) { + setString( + parameterIndex, + StringUtils.fixDecimalExponent( + StringUtils.consistentToString((BigDecimal) parameterObj))); + } else { + setString(parameterIndex, parameterObj.toString()); + } + + break; + + case Types.CLOB: + if (parameterObj instanceof java.sql.Clob) { + setClob(parameterIndex, (java.sql.Clob) parameterObj); + } else { + setString(parameterIndex, parameterObj.toString()); + } + + break; + + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + case Types.BLOB: + throw new SQLException(Constant.PARAMETER_SUPPORTED); + case Types.DATE: + case Types.TIMESTAMP: + java.util.Date parameterAsDate; + + if (parameterObj instanceof String) { + ParsePosition pp = new ParsePosition(0); + DateFormat sdf = + new SimpleDateFormat(getDateTimePattern((String) parameterObj, false), Locale.US); + parameterAsDate = sdf.parse((String) parameterObj, pp); + } else { + parameterAsDate = (Date) parameterObj; + } + + switch (targetSqlType) { + case Types.DATE: + if (parameterAsDate instanceof java.sql.Date) { + setDate(parameterIndex, (java.sql.Date) parameterAsDate); + } else { + setDate(parameterIndex, new java.sql.Date(parameterAsDate.getTime())); + } + + break; + + case Types.TIMESTAMP: + if (parameterAsDate instanceof java.sql.Timestamp) { + setTimestamp(parameterIndex, (java.sql.Timestamp) parameterAsDate); + } else { + setTimestamp(parameterIndex, new java.sql.Timestamp(parameterAsDate.getTime())); + } + + break; + default: + logger.error("No type was matched"); + break; + } + + break; + + case Types.TIME: + if (parameterObj instanceof String) { + DateFormat sdf = + new SimpleDateFormat(getDateTimePattern((String) parameterObj, true), Locale.US); + setTime(parameterIndex, new Time(sdf.parse((String) parameterObj).getTime())); + } else if (parameterObj instanceof Timestamp) { + Timestamp xT = (Timestamp) parameterObj; + setTime(parameterIndex, new Time(xT.getTime())); + } else { + setTime(parameterIndex, (Time) parameterObj); + } + + break; + + case Types.OTHER: + throw new SQLException(Constant.PARAMETER_SUPPORTED); // + default: + throw new SQLException(Constant.PARAMETER_SUPPORTED); // + } + } catch (SQLException ex) { + throw ex; + } catch (Exception ex) { + throw new SQLException(Constant.PARAMETER_SUPPORTED); // + } + } + } + + @SuppressWarnings({ + "squid:S3776", + "squid:S6541" + }) // ignore Cognitive Complexity of methods should not be too high + // ignore Methods should not perform too many tasks (aka Brain method) + private final String getDateTimePattern(String dt, boolean toTime) throws Exception { + // + // Special case + // + int dtLength = (dt != null) ? dt.length() : 0; + + if ((dtLength >= 8) && (dtLength <= 10)) { + int dashCount = 0; + boolean isDateOnly = true; + + for (int i = 0; i < dtLength; i++) { + char c = dt.charAt(i); + + if (!Character.isDigit(c) && (c != '-')) { + isDateOnly = false; + + break; + } + + if (c == '-') { + dashCount++; + } + } + + if (isDateOnly && (dashCount == 2)) { + return "yyyy-MM-dd"; + } + } + boolean colonsOnly = true; + + for (int i = 0; i < dtLength; i++) { + char c = dt.charAt(i); + + if (!Character.isDigit(c) && (c != ':')) { + colonsOnly = false; + + break; + } + } + + if (colonsOnly) { + return "HH:mm:ss"; + } + + int n; + int z; + int count; + int maxvecs; + char c; + char separator; + StringReader reader = new StringReader(dt + " "); + ArrayList vec = new ArrayList<>(); + ArrayList vecRemovelist = new ArrayList<>(); + Object[] nv = new Object[3]; + Object[] v; + nv[0] = Character.valueOf('y'); + nv[1] = new StringBuilder(); + nv[2] = Integer.valueOf(0); + vec.add(nv); + + if (toTime) { + nv = new Object[3]; + nv[0] = Character.valueOf('h'); + nv[1] = new StringBuilder(); + nv[2] = Integer.valueOf(0); + vec.add(nv); + } + + while ((z = reader.read()) != -1) { + separator = (char) z; + maxvecs = vec.size(); + + for (count = 0; count < maxvecs; count++) { + v = vec.get(count); + n = ((Integer) v[2]).intValue(); + c = getSuccessor(((Character) v[0]).charValue(), n); + + if (!Character.isLetterOrDigit(separator)) { + if ((c == ((Character) v[0]).charValue()) && (c != 'S')) { + vecRemovelist.add(v); + } else { + ((StringBuilder) v[1]).append(separator); + + if ((c == 'X') || (c == 'Y')) { + v[2] = Integer.valueOf(4); + } + } + } else { + if (c == 'X') { + c = 'y'; + nv = new Object[3]; + nv[1] = (new StringBuilder(((StringBuilder) v[1]).toString())).append('M'); + nv[0] = Character.valueOf('M'); + nv[2] = Integer.valueOf(1); + vec.add(nv); + } else if (c == 'Y') { + c = 'M'; + nv = new Object[3]; + nv[1] = (new StringBuilder(((StringBuilder) v[1]).toString())).append('d'); + nv[0] = Character.valueOf('d'); + nv[2] = Integer.valueOf(1); + vec.add(nv); + } + + ((StringBuilder) v[1]).append(c); + + if (c == ((Character) v[0]).charValue()) { + v[2] = Integer.valueOf(n + 1); + } else { + v[0] = Character.valueOf(c); + v[2] = Integer.valueOf(1); + } + } + } + + int size = vecRemovelist.size(); + + for (int i = 0; i < size; i++) { + v = vecRemovelist.get(i); + vec.remove(v); + } + + vecRemovelist.clear(); + } + + int size = vec.size(); + + for (int i = 0; i < size; i++) { + v = vec.get(i); + c = ((Character) v[0]).charValue(); + n = ((Integer) v[2]).intValue(); + + boolean bk = getSuccessor(c, n) != c; + boolean atEnd = (((c == 's') || (c == 'm') || ((c == 'h') && toTime)) && bk); + boolean finishesAtDate = (bk && (c == 'd') && !toTime); + boolean containsEnd = (((StringBuilder) v[1]).toString().indexOf('W') != -1); + + if ((!atEnd && !finishesAtDate) || (containsEnd)) { + vecRemovelist.add(v); + } + } + + size = vecRemovelist.size(); + + for (int i = 0; i < size; i++) { + vec.remove(vecRemovelist.get(i)); + } + + vecRemovelist.clear(); + v = vec.get(0); // might throw exception + + StringBuilder format = (StringBuilder) v[1]; + format.setLength(format.length() - 1); + + return format.toString(); + } + + @SuppressWarnings({"squid:S3776", "squid:S3358"}) // ignore Ternary operators should not be nested + // ignore Cognitive Complexity of methods should not be too high + private final char getSuccessor(char c, int n) { + return ((c == 'y') && (n == 2)) + ? 'X' + : (((c == 'y') && (n < 4)) + ? 'y' + : ((c == 'y') + ? 'M' + : (((c == 'M') && (n == 2)) + ? 'Y' + : (((c == 'M') && (n < 3)) + ? 'M' + : ((c == 'M') + ? 'd' + : (((c == 'd') && (n < 2)) + ? 'd' + : ((c == 'd') + ? 'H' + : (((c == 'H') && (n < 2)) + ? 'H' + : ((c == 'H') + ? 'm' + : (((c == 'm') && (n < 2)) + ? 'm' + : ((c == 'm') + ? 's' + : (((c == 's') && (n < 2)) + ? 's' + : 'W')))))))))))); + } + + @SuppressWarnings({ + "squid:S3776", + "squid:S6541" + }) // ignore Cognitive Complexity of methods should not be too high + // ignore Methods should not perform too many tasks (aka Brain method) + private void setNumericObject( + int parameterIndex, Object parameterObj, int targetSqlType, int scale) throws SQLException { + Number parameterAsNum; + + if (parameterObj instanceof Boolean) { + parameterAsNum = + ((Boolean) parameterObj).booleanValue() ? Integer.valueOf(1) : Integer.valueOf(0); + } else if (parameterObj instanceof String) { + switch (targetSqlType) { + case Types.BIT: + if ("1".equals(parameterObj) || "0".equals(parameterObj)) { + parameterAsNum = Integer.valueOf((String) parameterObj); + } else { + boolean parameterAsBoolean = "true".equalsIgnoreCase((String) parameterObj); + + parameterAsNum = parameterAsBoolean ? Integer.valueOf(1) : Integer.valueOf(0); + } + + break; + + case Types.TINYINT: + case Types.SMALLINT: + case Types.INTEGER: + parameterAsNum = Integer.valueOf((String) parameterObj); + + break; + + case Types.BIGINT: + parameterAsNum = Long.valueOf((String) parameterObj); + + break; + + case Types.REAL: + parameterAsNum = Float.valueOf((String) parameterObj); + + break; + + case Types.FLOAT: + case Types.DOUBLE: + parameterAsNum = Double.valueOf((String) parameterObj); + + break; + + case Types.DECIMAL: + case Types.NUMERIC: + default: + parameterAsNum = new java.math.BigDecimal((String) parameterObj); + } + } else { + parameterAsNum = (Number) parameterObj; + } + + switch (targetSqlType) { + case Types.BIT: + case Types.TINYINT: + case Types.SMALLINT: + case Types.INTEGER: + setInt(parameterIndex, parameterAsNum.intValue()); + break; + + case Types.BIGINT: + setLong(parameterIndex, parameterAsNum.longValue()); + break; + + case Types.REAL: + setFloat(parameterIndex, parameterAsNum.floatValue()); + break; + + case Types.FLOAT: + setFloat(parameterIndex, parameterAsNum.floatValue()); + break; + case Types.DOUBLE: + setDouble(parameterIndex, parameterAsNum.doubleValue()); + + break; + + case Types.DECIMAL: + case Types.NUMERIC: + if (parameterAsNum instanceof java.math.BigDecimal) { + BigDecimal scaledBigDecimal = null; + + try { + scaledBigDecimal = ((java.math.BigDecimal) parameterAsNum).setScale(scale); + } catch (ArithmeticException ex) { + try { + scaledBigDecimal = + ((java.math.BigDecimal) parameterAsNum).setScale(scale, BigDecimal.ROUND_HALF_UP); + } catch (ArithmeticException arEx) { + throw new SQLException( + "Can't set scale of '" + + scale + + "' for DECIMAL argument '" + + parameterAsNum + + "'"); + } + } + + setBigDecimal(parameterIndex, scaledBigDecimal); + } else if (parameterAsNum instanceof java.math.BigInteger) { + setBigDecimal( + parameterIndex, + new java.math.BigDecimal((java.math.BigInteger) parameterAsNum, scale)); + } else { + setBigDecimal(parameterIndex, BigDecimal.valueOf(parameterAsNum.doubleValue())); + } + + break; + default: + } } @Override - public void setNClob(int parameterIndex, NClob value) throws SQLException { + public void setRef(int parameterIndex, Ref x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setNClob(int parameterIndex, Reader reader) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setRowId(int parameterIndex, RowId x) throws SQLException { + throw new SQLException(METHOD_NOT_SUPPORTED_STRING); } @Override - public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { + throw new SQLException(METHOD_NOT_SUPPORTED_STRING); } @Override - public void setNString(int parameterIndex, String value) throws SQLException { + public void setShort(int parameterIndex, short x) throws SQLException { throw new SQLException(Constant.PARAMETER_SUPPORTED); } @Override - public void setRef(int parameterIndex, Ref x) throws SQLException { - throw new SQLException(Constant.PARAMETER_SUPPORTED); + public void setString(int parameterIndex, String x) { + if (x == null) { + this.parameters.put(parameterIndex, null); + } else { + this.parameters.put(parameterIndex, "'" + escapeSingleQuotes(x) + "'"); + } + } + + private String escapeSingleQuotes(String value) { + // Escape single quotes with double single quotes + return value.replace("'", "''"); } @Override - public void setRowId(int parameterIndex, RowId x) throws SQLException { - throw new SQLException(METHOD_NOT_SUPPORTED_STRING); + public void setTime(int parameterIndex, Time x) throws SQLException { + try { + long time = x.getTime(); + String timeprecision = client.getProperties().getTimestampPrecision(); + switch (timeprecision.toLowerCase()) { + case "ms": + break; + case "us": + time = time * 1000; + break; + case "ns": + time = time * 1000000; + break; + default: + break; + } + setLong(parameterIndex, time); + } catch (TException e) { + logger.error( + String.format("set time error when iotdb prepared statement :%s ", e.getMessage())); + } } @Override - public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { - throw new SQLException(METHOD_NOT_SUPPORTED_STRING); + public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { + try { + ZonedDateTime zonedDateTime = null; + long time = x.getTime(); + String timeprecision = client.getProperties().getTimestampPrecision(); + switch (timeprecision.toLowerCase()) { + case "ms": + break; + case "us": + time = time * 1000; + break; + case "ns": + time = time * 1000000; + break; + default: + break; + } + if (cal != null) { + zonedDateTime = + ZonedDateTime.ofInstant( + Instant.ofEpochMilli(time), ZoneId.of(cal.getTimeZone().getID())); + } else { + zonedDateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(time), super.zoneId); + } + this.parameters.put( + parameterIndex, zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + } catch (TException e) { + logger.error( + String.format("set time error when iotdb prepared statement :%s ", e.getMessage())); + } } @Override - public void setShort(int parameterIndex, short x) throws SQLException { - setInt(parameterIndex, x); + public void setTimestamp(int parameterIndex, Timestamp x) { + ZonedDateTime zonedDateTime = + ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()), super.zoneId); + this.parameters.put( + parameterIndex, zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + } + + @Override + public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { + ZonedDateTime zonedDateTime = null; + if (cal != null) { + zonedDateTime = + ZonedDateTime.ofInstant( + Instant.ofEpochMilli(x.getTime()), ZoneId.of(cal.getTimeZone().getID())); + } else { + zonedDateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()), super.zoneId); + } + this.parameters.put( + parameterIndex, zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); } @Override @@ -753,14 +1010,16 @@ public void setUnicodeStream(int parameterIndex, InputStream x, int length) thro throw new SQLException(Constant.PARAMETER_SUPPORTED); } - // ================== Helper Methods for Backward Compatibility ================== - private String createCompleteSql(final String sql, Map parameters) throws SQLException { List parts = splitSqlStatement(sql); StringBuilder newSql = new StringBuilder(parts.get(0)); for (int i = 1; i < parts.size(); i++) { + if (logger.isDebugEnabled()) { + logger.debug("SQL {}", sql); + logger.debug("parameters {}", parameters.size()); + } if (!parameters.containsKey(i)) { throw new SQLException("Parameter #" + i + " is unset"); } @@ -784,12 +1043,15 @@ private List splitSqlStatement(final String sql) { } switch (c) { case '\'': + // skip something like 'xxxxx' apCount++; break; case '\\': + // skip something like \r\n skip = true; break; case '?': + // for input like: select a from 'bc' where d, 'bc' will be skipped if ((apCount & 1) == 0) { parts.add(sql.substring(off, i)); off = i + 1; diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java new file mode 100644 index 000000000000..4bcf008f29bf --- /dev/null +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java @@ -0,0 +1,722 @@ +/* + * 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.iotdb.jdbc; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.StatementExecutionException; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.rpc.stmt.PreparedParameterSerializer; +import org.apache.iotdb.service.rpc.thrift.IClientRPCService.Iface; +import org.apache.iotdb.service.rpc.thrift.TSDeallocatePreparedReq; +import org.apache.iotdb.service.rpc.thrift.TSExecutePreparedReq; +import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp; +import org.apache.iotdb.service.rpc.thrift.TSPrepareReq; +import org.apache.iotdb.service.rpc.thrift.TSPrepareResp; + +import org.apache.thrift.TException; +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.io.Reader; +import java.math.BigDecimal; +import java.net.URL; +import java.nio.charset.Charset; +import java.sql.Array; +import java.sql.Blob; +import java.sql.Clob; +import java.sql.Date; +import java.sql.NClob; +import java.sql.ParameterMetaData; +import java.sql.PreparedStatement; +import java.sql.Ref; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.RowId; +import java.sql.SQLException; +import java.sql.SQLXML; +import java.sql.Time; +import java.sql.Timestamp; +import java.sql.Types; +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +public class IoTDBTablePreparedStatement extends IoTDBStatement implements PreparedStatement { + + private static final Logger logger = LoggerFactory.getLogger(IoTDBTablePreparedStatement.class); + private static final String METHOD_NOT_SUPPORTED_STRING = "Method not supported"; + + private final String sql; + private final String preparedStatementName; + private final int parameterCount; + + // Parameter values stored as objects for binary serialization + private final Object[] parameterValues; + private final int[] parameterTypes; + + /** save the SQL parameters as (paramLoc,paramValue) pairs for backward compatibility. */ + private final Map parameters = new HashMap<>(); + + IoTDBTablePreparedStatement( + IoTDBConnection connection, + Iface client, + Long sessionId, + String sql, + ZoneId zoneId, + Charset charset) + throws SQLException { + super(connection, client, sessionId, zoneId, charset); + this.sql = sql; + this.preparedStatementName = generateStatementName(); + + // Send PREPARE request to server + TSPrepareReq prepareReq = new TSPrepareReq(); + prepareReq.setSessionId(sessionId); + prepareReq.setSql(sql); + prepareReq.setStatementName(preparedStatementName); + + try { + TSPrepareResp resp = client.prepareStatement(prepareReq); + RpcUtils.verifySuccess(resp.getStatus()); + + this.parameterCount = resp.isSetParameterCount() ? resp.getParameterCount() : 0; + this.parameterValues = new Object[parameterCount]; + this.parameterTypes = new int[parameterCount]; + + // Initialize all parameter types to NULL + for (int i = 0; i < parameterCount; i++) { + parameterTypes[i] = Types.NULL; + } + } catch (TException e) { + throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); + } catch (StatementExecutionException e) { + throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); + } + } + + // Only for tests + IoTDBTablePreparedStatement( + IoTDBConnection connection, Iface client, Long sessionId, String sql, ZoneId zoneId) + throws SQLException { + this(connection, client, sessionId, sql, zoneId, TSFileConfig.STRING_CHARSET); + } + + private String generateStatementName() { + return "jdbc_ps_" + UUID.randomUUID().toString().replace("-", ""); + } + + @Override + public void addBatch() throws SQLException { + super.addBatch(createCompleteSql(sql, parameters)); + } + + @Override + public void clearParameters() { + this.parameters.clear(); + for (int i = 0; i < parameterCount; i++) { + parameterValues[i] = null; + parameterTypes[i] = Types.NULL; + } + } + + @Override + public boolean execute() throws SQLException { + TSExecuteStatementResp resp = executeInternal(); + return resp.isSetQueryDataSet() || resp.isSetQueryResult(); + } + + @Override + public ResultSet executeQuery() throws SQLException { + TSExecuteStatementResp resp = executeInternal(); + return processQueryResult(resp); + } + + @Override + public int executeUpdate() throws SQLException { + executeInternal(); + return 0; // IoTDB doesn't return affected row count + } + + private TSExecuteStatementResp executeInternal() throws SQLException { + // Validate all parameters are set + for (int i = 0; i < parameterCount; i++) { + if (parameterTypes[i] == Types.NULL + && parameterValues[i] == null + && !parameters.containsKey(i + 1)) { + throw new SQLException("Parameter #" + (i + 1) + " is unset"); + } + } + + TSExecutePreparedReq req = new TSExecutePreparedReq(); + req.setSessionId(sessionId); + req.setStatementName(preparedStatementName); + req.setParameters( + PreparedParameterSerializer.serialize(parameterValues, parameterTypes, parameterCount)); + req.setStatementId(getStmtId()); + + if (queryTimeout > 0) { + req.setTimeout(queryTimeout * 1000L); + } + + try { + TSExecuteStatementResp resp = client.executePreparedStatement(req); + RpcUtils.verifySuccess(resp.getStatus()); + return resp; + } catch (TException e) { + throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); + } catch (StatementExecutionException e) { + throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); + } + } + + private ResultSet processQueryResult(TSExecuteStatementResp resp) throws SQLException { + if (resp.isSetQueryDataSet() || resp.isSetQueryResult()) { + // Create ResultSet from response + this.resultSet = + new IoTDBJDBCResultSet( + this, + resp.getColumns(), + resp.getDataTypeList(), + resp.columnNameIndexMap, + resp.ignoreTimeStamp, + client, + sql, + resp.queryId, + sessionId, + resp.queryResult, + resp.tracingInfo, + (long) queryTimeout * 1000, + resp.isSetMoreData() && resp.isMoreData(), + zoneId); + return resultSet; + } + return null; + } + + @Override + public void close() throws SQLException { + if (!isClosed()) { + // Deallocate prepared statement on server + TSDeallocatePreparedReq req = new TSDeallocatePreparedReq(); + req.setSessionId(sessionId); + req.setStatementName(preparedStatementName); + + try { + TSStatus status = client.deallocatePreparedStatement(req); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + logger.warn("Failed to deallocate prepared statement: {}", status.getMessage()); + } + } catch (TException e) { + logger.warn("Error deallocating prepared statement", e); + } + } + super.close(); + } + + @Override + public ResultSetMetaData getMetaData() throws SQLException { + if (resultSet != null) { + return resultSet.getMetaData(); + } + return null; + } + + @Override + public ParameterMetaData getParameterMetaData() { + return new ParameterMetaData() { + @Override + public int getParameterCount() { + return parameterCount; + } + + @Override + public int isNullable(int param) { + return ParameterMetaData.parameterNullableUnknown; + } + + @Override + public boolean isSigned(int param) { + int type = parameterTypes[param - 1]; + return type == Types.INTEGER + || type == Types.BIGINT + || type == Types.FLOAT + || type == Types.DOUBLE; + } + + @Override + public int getPrecision(int param) { + return 0; + } + + @Override + public int getScale(int param) { + return 0; + } + + @Override + public int getParameterType(int param) { + return parameterTypes[param - 1]; + } + + @Override + public String getParameterTypeName(int param) { + return null; + } + + @Override + public String getParameterClassName(int param) { + return null; + } + + @Override + public int getParameterMode(int param) { + return ParameterMetaData.parameterModeIn; + } + + @Override + public T unwrap(Class iface) { + return null; + } + + @Override + public boolean isWrapperFor(Class iface) { + return false; + } + }; + } + + // ================== Parameter Setters ================== + + @Override + public void setNull(int parameterIndex, int sqlType) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = null; + parameterTypes[parameterIndex - 1] = Types.NULL; + this.parameters.put(parameterIndex, "NULL"); + } + + @Override + public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException { + setNull(parameterIndex, sqlType); + } + + @Override + public void setBoolean(int parameterIndex, boolean x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.BOOLEAN; + this.parameters.put(parameterIndex, Boolean.toString(x)); + } + + @Override + public void setInt(int parameterIndex, int x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.INTEGER; + this.parameters.put(parameterIndex, Integer.toString(x)); + } + + @Override + public void setLong(int parameterIndex, long x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.BIGINT; + this.parameters.put(parameterIndex, Long.toString(x)); + } + + @Override + public void setFloat(int parameterIndex, float x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.FLOAT; + this.parameters.put(parameterIndex, Float.toString(x)); + } + + @Override + public void setDouble(int parameterIndex, double x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.DOUBLE; + this.parameters.put(parameterIndex, Double.toString(x)); + } + + @Override + public void setString(int parameterIndex, String x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.VARCHAR; + if (x == null) { + this.parameters.put(parameterIndex, null); + } else { + this.parameters.put(parameterIndex, "'" + escapeSingleQuotes(x) + "'"); + } + } + + @Override + public void setBytes(int parameterIndex, byte[] x) throws SQLException { + checkParameterIndex(parameterIndex); + parameterValues[parameterIndex - 1] = x; + parameterTypes[parameterIndex - 1] = Types.BINARY; + Binary binary = new Binary(x); + this.parameters.put(parameterIndex, binary.getStringValue(TSFileConfig.STRING_CHARSET)); + } + + @Override + public void setDate(int parameterIndex, Date x) throws SQLException { + checkParameterIndex(parameterIndex); + DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); + String dateStr = dateFormat.format(x); + parameterValues[parameterIndex - 1] = dateStr; + parameterTypes[parameterIndex - 1] = Types.VARCHAR; + this.parameters.put(parameterIndex, "'" + dateStr + "'"); + } + + @Override + public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { + setDate(parameterIndex, x); + } + + @Override + public void setTime(int parameterIndex, Time x) throws SQLException { + checkParameterIndex(parameterIndex); + try { + long time = x.getTime(); + String timeprecision = client.getProperties().getTimestampPrecision(); + switch (timeprecision.toLowerCase()) { + case "ms": + break; + case "us": + time = time * 1000; + break; + case "ns": + time = time * 1000000; + break; + default: + break; + } + parameterValues[parameterIndex - 1] = time; + parameterTypes[parameterIndex - 1] = Types.BIGINT; + this.parameters.put(parameterIndex, Long.toString(time)); + } catch (TException e) { + throw new SQLException("Failed to get time precision: " + e.getMessage(), e); + } + } + + @Override + public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { + setTime(parameterIndex, x); + } + + @Override + public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { + checkParameterIndex(parameterIndex); + ZonedDateTime zonedDateTime = + ZonedDateTime.ofInstant(Instant.ofEpochMilli(x.getTime()), super.zoneId); + String tsStr = zonedDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + parameterValues[parameterIndex - 1] = tsStr; + parameterTypes[parameterIndex - 1] = Types.VARCHAR; + this.parameters.put(parameterIndex, tsStr); + } + + @Override + public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { + setTimestamp(parameterIndex, x); + } + + @Override + public void setObject(int parameterIndex, Object x) throws SQLException { + if (x == null) { + setNull(parameterIndex, Types.NULL); + } else if (x instanceof String) { + setString(parameterIndex, (String) x); + } else if (x instanceof Integer) { + setInt(parameterIndex, (Integer) x); + } else if (x instanceof Long) { + setLong(parameterIndex, (Long) x); + } else if (x instanceof Float) { + setFloat(parameterIndex, (Float) x); + } else if (x instanceof Double) { + setDouble(parameterIndex, (Double) x); + } else if (x instanceof Boolean) { + setBoolean(parameterIndex, (Boolean) x); + } else if (x instanceof Timestamp) { + setTimestamp(parameterIndex, (Timestamp) x); + } else if (x instanceof Date) { + setDate(parameterIndex, (Date) x); + } else if (x instanceof Time) { + setTime(parameterIndex, (Time) x); + } else if (x instanceof byte[]) { + setBytes(parameterIndex, (byte[]) x); + } else { + throw new SQLException( + String.format( + "Can't infer the SQL type for an instance of %s. Use setObject() with explicit type.", + x.getClass().getName())); + } + } + + @Override + public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { + setObject(parameterIndex, x); + } + + @Override + public void setObject(int parameterIndex, Object parameterObj, int targetSqlType, int scale) + throws SQLException { + setObject(parameterIndex, parameterObj); + } + + private void checkParameterIndex(int index) throws SQLException { + if (index < 1 || index > parameterCount) { + throw new SQLException( + "Parameter index out of range: " + index + " (expected 1-" + parameterCount + ")"); + } + } + + private String escapeSingleQuotes(String value) { + return value.replace("'", "''"); + } + + // ================== Unsupported Methods ================== + + @Override + public void setArray(int parameterIndex, Array x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { + try { + byte[] bytes = ReadWriteIOUtils.readBytes(x, length); + setBytes(parameterIndex, bytes); + } catch (IOException e) { + throw new SQLException("Failed to read binary stream: " + e.getMessage(), e); + } + } + + @Override + public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setBlob(int parameterIndex, Blob x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setBlob(int parameterIndex, InputStream inputStream, long length) + throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setByte(int parameterIndex, byte x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setCharacterStream(int parameterIndex, Reader reader, int length) + throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setCharacterStream(int parameterIndex, Reader reader, long length) + throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setClob(int parameterIndex, Clob x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setClob(int parameterIndex, Reader reader) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setNCharacterStream(int parameterIndex, Reader value, long length) + throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setNClob(int parameterIndex, NClob value) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setNClob(int parameterIndex, Reader reader) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setNString(int parameterIndex, String value) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setRef(int parameterIndex, Ref x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setRowId(int parameterIndex, RowId x) throws SQLException { + throw new SQLException(METHOD_NOT_SUPPORTED_STRING); + } + + @Override + public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { + throw new SQLException(METHOD_NOT_SUPPORTED_STRING); + } + + @Override + public void setShort(int parameterIndex, short x) throws SQLException { + setInt(parameterIndex, x); + } + + @Override + public void setURL(int parameterIndex, URL x) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + @Override + public void setUnicodeStream(int parameterIndex, InputStream x, int length) throws SQLException { + throw new SQLException(Constant.PARAMETER_SUPPORTED); + } + + // ================== Helper Methods for Backward Compatibility ================== + + private String createCompleteSql(final String sql, Map parameters) + throws SQLException { + List parts = splitSqlStatement(sql); + + StringBuilder newSql = new StringBuilder(parts.get(0)); + for (int i = 1; i < parts.size(); i++) { + if (!parameters.containsKey(i)) { + throw new SQLException("Parameter #" + i + " is unset"); + } + newSql.append(parameters.get(i)); + newSql.append(parts.get(i)); + } + return newSql.toString(); + } + + private List splitSqlStatement(final String sql) { + List parts = new ArrayList<>(); + int apCount = 0; + int off = 0; + boolean skip = false; + + for (int i = 0; i < sql.length(); i++) { + char c = sql.charAt(i); + if (skip) { + skip = false; + continue; + } + switch (c) { + case '\'': + apCount++; + break; + case '\\': + skip = true; + break; + case '?': + if ((apCount & 1) == 0) { + parts.add(sql.substring(off, i)); + off = i + 1; + } + break; + default: + break; + } + } + parts.add(sql.substring(off)); + return parts; + } +} diff --git a/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java index aa932cda5f22..f80b8a83936c 100644 --- a/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java +++ b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBPreparedStatementTest.java @@ -22,27 +22,23 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.IClientRPCService.Iface; -import org.apache.iotdb.service.rpc.thrift.TSExecutePreparedReq; import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementReq; import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp; -import org.apache.iotdb.service.rpc.thrift.TSPrepareReq; -import org.apache.iotdb.service.rpc.thrift.TSPrepareResp; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import java.sql.SQLException; import java.sql.Timestamp; import java.sql.Types; import java.time.ZoneId; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -50,7 +46,6 @@ public class IoTDBPreparedStatementTest { @Mock TSExecuteStatementResp execStatementResp; - @Mock TSPrepareResp prepareResp; @Mock TSStatus getOperationStatusResp; private ZoneId zoneId = ZoneId.systemDefault(); @Mock private IoTDBConnection connection; @@ -67,53 +62,6 @@ public void before() throws Exception { when(execStatementResp.getQueryId()).thenReturn(queryId); when(client.executeStatementV2(any(TSExecuteStatementReq.class))).thenReturn(execStatementResp); - - // Mock for prepareStatement - dynamically calculate parameter count from SQL - when(client.prepareStatement(any(TSPrepareReq.class))) - .thenAnswer( - new Answer() { - @Override - public TSPrepareResp answer(InvocationOnMock invocation) throws Throwable { - TSPrepareReq req = invocation.getArgument(0); - String sql = req.getSql(); - int paramCount = countQuestionMarks(sql); - - TSPrepareResp resp = new TSPrepareResp(); - resp.setStatus(Status_SUCCESS); - resp.setParameterCount(paramCount); - return resp; - } - }); - - // Mock for executePreparedStatement - when(client.executePreparedStatement(any(TSExecutePreparedReq.class))) - .thenReturn(execStatementResp); - } - - /** Count the number of '?' placeholders in a SQL string, ignoring those inside quotes */ - private int countQuestionMarks(String sql) { - int count = 0; - boolean inSingleQuote = false; - boolean inDoubleQuote = false; - - for (int i = 0; i < sql.length(); i++) { - char c = sql.charAt(i); - - if (c == '\'' && !inDoubleQuote) { - // Check for escaped quote - if (i + 1 < sql.length() && sql.charAt(i + 1) == '\'') { - i++; // Skip escaped quote - } else { - inSingleQuote = !inSingleQuote; - } - } else if (c == '"' && !inSingleQuote) { - inDoubleQuote = !inDoubleQuote; - } else if (c == '?' && !inSingleQuote && !inDoubleQuote) { - count++; - } - } - - return count; } @SuppressWarnings("resource") @@ -125,27 +73,23 @@ public void testNonParameterized() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.execute(); - // Verify executePreparedStatement was called (new behavior) - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - // Non-parameterized query should have empty parameters - assertTrue( - argument.getValue().getParameters() == null - || argument.getValue().getParameters().isEmpty()); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @Test public void unusedArgument() throws SQLException { - // SQL with no parameters - setting a parameter should throw an exception String sql = "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 24 and time > 2017-11-1 0:13:00"; IoTDBPreparedStatement ps = new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - // In the new server-side prepared statement implementation, setting a parameter - // that doesn't exist in the SQL throws an exception - assertThrows(SQLException.class, () -> ps.setString(1, "123")); + ps.setString(1, "123"); + assertFalse(ps.execute()); } @SuppressWarnings("resource") @@ -167,11 +111,12 @@ public void oneIntArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setInt(1, 123); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - // Verify parameters were sent - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123 and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -183,10 +128,12 @@ public void oneLongArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setLong(1, 123); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123 and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -198,10 +145,12 @@ public void oneFloatArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setFloat(1, 123.133f); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123.133 and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -213,10 +162,12 @@ public void oneDoubleArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setDouble(1, 123.456); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < 123.456 and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -228,10 +179,12 @@ public void oneBooleanArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setBoolean(1, false); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < false and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -243,10 +196,12 @@ public void oneStringArgument1() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setString(1, "'abcde'"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < '''abcde''' and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -258,10 +213,12 @@ public void oneStringArgument2() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setString(1, "\"abcde\""); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE temperature < '\"abcde\"' and time > 2017-11-1 0:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -272,10 +229,11 @@ public void oneStringArgument3() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setString(1, "temperature"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, 'temperature' FROM root.ln.wf01.wt01", argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -286,10 +244,12 @@ public void oneTimeLongArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setLong(1, 1233); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE time > 1233", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -300,10 +260,12 @@ public void oneTimeTimestampArgument() throws Exception { new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); ps.setTimestamp(1, Timestamp.valueOf("2017-11-01 00:13:00")); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE time > 2017-11-01T00:13:00", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -316,10 +278,12 @@ public void escapingOfStringArgument() throws Exception { ps.setLong(1, 1333); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE status = '134' and temperature = 1333", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -333,10 +297,12 @@ public void pastingIntoEscapedQuery() throws Exception { ps.setDouble(1, -1323.0); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "SELECT status, temperature FROM root.ln.wf01.wt01 WHERE status = '\\044e' || temperature = -1323.0", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -355,10 +321,12 @@ public void testInsertStatement1() throws Exception { ps.setString(7, "'abc'"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f) VALUES(12324,false,123,123234345,123.423,-1323.0,'''abc''')", + argument.getValue().getStatement()); } @SuppressWarnings("resource") @@ -379,10 +347,12 @@ public void testInsertStatement2() throws Exception { ps.setString(9, "'abc'"); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "INSERT INTO root.ln.wf01.wt01(time,a,b,c,d,e,f,g,h) VALUES(2017-11-01T00:13:00,false,123,123234345,123.423,-1323.0,'\"abc\"','abc','''abc''')", + argument.getValue().getStatement()); } @Test @@ -400,10 +370,12 @@ public void testInsertStatement3() throws Exception { ps.setObject(7, "\"abc\"", Types.VARCHAR); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "INSERT INTO root.ln.wf01.wt02(time,a,b,c,d,e,f) VALUES(2020-01-01T10:10:10,false,123,123234345,123.423,-1323.0,'\"abc\"')", + argument.getValue().getStatement()); } @Test @@ -421,146 +393,11 @@ public void testInsertStatement4() throws Exception { ps.setObject(7, "abc", Types.VARCHAR); ps.execute(); - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); - } - - // ========== Table Model SQL Injection Prevention Tests ========== - - @SuppressWarnings("resource") - @Test - public void testTableModelLoginInjectionWithComment() throws Exception { - // Login interface SQL injection attack 1: Using -- comments to bypass password checks - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE username = ? AND password = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "admin' --"); - ps.setString(2, "password"); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - // SQL injection is prevented by using prepared statements with parameterized queries - assertTrue(argument.getValue().getParameters() != null); - } - - @SuppressWarnings("resource") - @Test - public void testTableModelLoginInjectionWithORCondition() throws Exception { - // Login interface SQL injection attack 2: Bypassing authentication by using 'OR '1'='1 - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE username = ? AND password = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "admin"); - ps.setString(2, "' OR '1'='1"); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - // SQL injection is prevented by using prepared statements with parameterized queries - assertTrue(argument.getValue().getParameters() != null); - } - - @SuppressWarnings("resource") - @Test - public void testTableModelQueryWithMultipleInjectionVectors() throws Exception { - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE email = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "'; DROP TABLE users;"); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - // SQL injection is prevented by using prepared statements with parameterized queries - assertTrue(argument.getValue().getParameters() != null); - } - - @SuppressWarnings("resource") - @Test - public void testTableModelString1() throws Exception { - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE password = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "a'b"); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); - } - - @SuppressWarnings("resource") - @Test - public void testTableModelString2() throws Exception { - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE password = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "a\'b"); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); - } - - @SuppressWarnings("resource") - @Test - public void testTableModelString3() throws Exception { - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE password = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "a\\'b"); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); - } - - @SuppressWarnings("resource") - @Test - public void testTableModelString4() throws Exception { - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE password = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, "a\\\'b"); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); - } - - @SuppressWarnings("resource") - @Test - public void testTableModelStringWithNull() throws Exception { - when(connection.getSqlDialect()).thenReturn("table"); - String sql = "SELECT * FROM users WHERE email = ?"; - IoTDBPreparedStatement ps = - new IoTDBPreparedStatement(connection, client, sessionId, sql, zoneId); - ps.setString(1, null); - ps.execute(); - - ArgumentCaptor argument = - ArgumentCaptor.forClass(TSExecutePreparedReq.class); - verify(client).executePreparedStatement(argument.capture()); - assertTrue(argument.getValue().getParameters() != null); + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecuteStatementReq.class); + verify(client).executeStatementV2(argument.capture()); + assertEquals( + "INSERT INTO root.ln.wf01.wt02(time,a,b,c,d,e,f) VALUES(2020-01-01T10:10:10,false,123,123234345,123.423,-1323.0,'abc')", + argument.getValue().getStatement()); } } diff --git a/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatementTest.java b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatementTest.java new file mode 100644 index 000000000000..dd1c7ecbcb20 --- /dev/null +++ b/iotdb-client/jdbc/src/test/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatementTest.java @@ -0,0 +1,240 @@ +/* + * 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.iotdb.jdbc; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.IClientRPCService.Iface; +import org.apache.iotdb.service.rpc.thrift.TSExecutePreparedReq; +import org.apache.iotdb.service.rpc.thrift.TSExecuteStatementResp; +import org.apache.iotdb.service.rpc.thrift.TSPrepareReq; +import org.apache.iotdb.service.rpc.thrift.TSPrepareResp; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.time.ZoneId; + +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class IoTDBTablePreparedStatementTest { + + @Mock TSExecuteStatementResp execStatementResp; + @Mock TSStatus getOperationStatusResp; + private ZoneId zoneId = ZoneId.systemDefault(); + @Mock private IoTDBConnection connection; + @Mock private Iface client; + @Mock private TSStatus successStatus = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + private TSStatus Status_SUCCESS = new TSStatus(successStatus); + private long queryId; + private long sessionId; + + @Before + public void before() throws Exception { + MockitoAnnotations.initMocks(this); + when(connection.getSqlDialect()).thenReturn("table"); + when(execStatementResp.getStatus()).thenReturn(Status_SUCCESS); + when(execStatementResp.getQueryId()).thenReturn(queryId); + + // Mock for prepareStatement - dynamically calculate parameter count from SQL + when(client.prepareStatement(any(TSPrepareReq.class))) + .thenAnswer( + new Answer() { + @Override + public TSPrepareResp answer(InvocationOnMock invocation) throws Throwable { + TSPrepareReq req = invocation.getArgument(0); + String sql = req.getSql(); + int paramCount = countQuestionMarks(sql); + + TSPrepareResp resp = new TSPrepareResp(); + resp.setStatus(Status_SUCCESS); + resp.setParameterCount(paramCount); + return resp; + } + }); + + // Mock for executePreparedStatement + when(client.executePreparedStatement(any(TSExecutePreparedReq.class))) + .thenReturn(execStatementResp); + } + + /** Count the number of '?' placeholders in a SQL string, ignoring those inside quotes */ + private int countQuestionMarks(String sql) { + int count = 0; + boolean inSingleQuote = false; + boolean inDoubleQuote = false; + + for (int i = 0; i < sql.length(); i++) { + char c = sql.charAt(i); + + if (c == '\'' && !inDoubleQuote) { + // Check for escaped quote + if (i + 1 < sql.length() && sql.charAt(i + 1) == '\'') { + i++; // Skip escaped quote + } else { + inSingleQuote = !inSingleQuote; + } + } else if (c == '"' && !inSingleQuote) { + inDoubleQuote = !inDoubleQuote; + } else if (c == '?' && !inSingleQuote && !inDoubleQuote) { + count++; + } + } + + return count; + } + + // ========== Table Model SQL Injection Prevention Tests ========== + + @SuppressWarnings("resource") + @Test + public void testTableModelLoginInjectionWithComment() throws Exception { + // Login interface SQL injection attack 1: Using -- comments to bypass password checks + String sql = "SELECT * FROM users WHERE username = ? AND password = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "admin' --"); + ps.setString(2, "password"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // SQL injection is prevented by using prepared statements with parameterized queries + assertTrue(argument.getValue().getParameters() != null); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelLoginInjectionWithORCondition() throws Exception { + // Login interface SQL injection attack 2: Bypassing authentication by using 'OR '1'='1 + String sql = "SELECT * FROM users WHERE username = ? AND password = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "admin"); + ps.setString(2, "' OR '1'='1"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // SQL injection is prevented by using prepared statements with parameterized queries + assertTrue(argument.getValue().getParameters() != null); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelQueryWithMultipleInjectionVectors() throws Exception { + String sql = "SELECT * FROM users WHERE email = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "'; DROP TABLE users;"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + // SQL injection is prevented by using prepared statements with parameterized queries + assertTrue(argument.getValue().getParameters() != null); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString1() throws Exception { + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString2() throws Exception { + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a\'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString3() throws Exception { + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a\\'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelString4() throws Exception { + String sql = "SELECT * FROM users WHERE password = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, "a\\\'b"); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); + } + + @SuppressWarnings("resource") + @Test + public void testTableModelStringWithNull() throws Exception { + String sql = "SELECT * FROM users WHERE email = ?"; + IoTDBTablePreparedStatement ps = + new IoTDBTablePreparedStatement(connection, client, sessionId, sql, zoneId); + ps.setString(1, null); + ps.execute(); + + ArgumentCaptor argument = + ArgumentCaptor.forClass(TSExecutePreparedReq.class); + verify(client).executePreparedStatement(argument.capture()); + assertTrue(argument.getValue().getParameters() != null); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java new file mode 100644 index 000000000000..81d25bc773b7 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java @@ -0,0 +1,200 @@ +/* + * 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.iotdb.rpc.stmt; + +import org.apache.tsfile.enums.TSDataType; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.sql.Types; +import java.util.ArrayList; +import java.util.List; + +/** + * Serializer for PreparedStatement parameters. + * + *

Binary format: [paramCount:4bytes][param1][param2]... + * + *

Each parameter: [type:1byte][value:variable] + */ +public class PreparedParameterSerializer { + + /** Deserialized parameter holding type and value. */ + public static class DeserializedParam { + public final TSDataType type; + public final Object value; + + DeserializedParam(TSDataType type, Object value) { + this.type = type; + this.value = value; + } + + public boolean isNull() { + return type == TSDataType.UNKNOWN || value == null; + } + } + + private PreparedParameterSerializer() {} + + // ================== Serialize (Client Side) ================== + + /** + * Serialize parameters to binary format. + * + * @param values parameter values + * @param jdbcTypes JDBC type codes (java.sql.Types) + * @param count number of parameters + * @return ByteBuffer containing serialized parameters + */ + public static ByteBuffer serialize(Object[] values, int[] jdbcTypes, int count) { + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + + dos.writeInt(count); + for (int i = 0; i < count; i++) { + serializeParameter(dos, values[i], jdbcTypes[i]); + } + + dos.flush(); + return ByteBuffer.wrap(baos.toByteArray()); + } catch (IOException e) { + throw new RuntimeException("Failed to serialize parameters", e); + } + } + + private static void serializeParameter(DataOutputStream dos, Object value, int jdbcType) + throws IOException { + if (value == null || jdbcType == Types.NULL) { + dos.writeByte(TSDataType.UNKNOWN.serialize()); + return; + } + + switch (jdbcType) { + case Types.BOOLEAN: + dos.writeByte(TSDataType.BOOLEAN.serialize()); + dos.writeByte((Boolean) value ? 1 : 0); + break; + + case Types.INTEGER: + dos.writeByte(TSDataType.INT32.serialize()); + dos.writeInt(((Number) value).intValue()); + break; + + case Types.BIGINT: + dos.writeByte(TSDataType.INT64.serialize()); + dos.writeLong(((Number) value).longValue()); + break; + + case Types.FLOAT: + dos.writeByte(TSDataType.FLOAT.serialize()); + dos.writeFloat(((Number) value).floatValue()); + break; + + case Types.DOUBLE: + dos.writeByte(TSDataType.DOUBLE.serialize()); + dos.writeDouble(((Number) value).doubleValue()); + break; + + case Types.VARCHAR: + case Types.CHAR: + byte[] strBytes = ((String) value).getBytes(StandardCharsets.UTF_8); + dos.writeByte(TSDataType.STRING.serialize()); + dos.writeInt(strBytes.length); + dos.write(strBytes); + break; + + case Types.BINARY: + case Types.VARBINARY: + byte[] binBytes = (byte[]) value; + dos.writeByte(TSDataType.BLOB.serialize()); + dos.writeInt(binBytes.length); + dos.write(binBytes); + break; + + default: + byte[] defaultBytes = String.valueOf(value).getBytes(StandardCharsets.UTF_8); + dos.writeByte(TSDataType.STRING.serialize()); + dos.writeInt(defaultBytes.length); + dos.write(defaultBytes); + break; + } + } + + // ================== Deserialize (Server Side) ================== + + /** + * Deserialize parameters from binary format. + * + * @param buffer ByteBuffer containing serialized parameters + * @return list of deserialized parameters with type and value + */ + public static List deserialize(ByteBuffer buffer) { + if (buffer == null || buffer.remaining() == 0) { + return new ArrayList<>(); + } + + buffer.rewind(); + int count = buffer.getInt(); + List result = new ArrayList<>(count); + + for (int i = 0; i < count; i++) { + byte typeCode = buffer.get(); + TSDataType type = TSDataType.deserialize(typeCode); + Object value = deserializeValue(buffer, type); + result.add(new DeserializedParam(type, value)); + } + + return result; + } + + private static Object deserializeValue(ByteBuffer buffer, TSDataType type) { + switch (type) { + case UNKNOWN: + return null; + case BOOLEAN: + return buffer.get() != 0; + case INT32: + return buffer.getInt(); + case INT64: + return buffer.getLong(); + case FLOAT: + return buffer.getFloat(); + case DOUBLE: + return buffer.getDouble(); + case TEXT: + case STRING: + int strLen = buffer.getInt(); + byte[] strBytes = new byte[strLen]; + buffer.get(strBytes); + return new String(strBytes, StandardCharsets.UTF_8); + case BLOB: + int binLen = buffer.getInt(); + byte[] binBytes = new byte[binLen]; + buffer.get(binBytes); + return binBytes; + default: + throw new IllegalArgumentException("Unsupported type: " + type); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index b8248801f741..efbd972f645e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -139,6 +139,8 @@ import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.rpc.stmt.PreparedParameterSerializer; +import org.apache.iotdb.rpc.stmt.PreparedParameterSerializer.DeserializedParam; import org.apache.iotdb.service.rpc.thrift.ServerProperties; import org.apache.iotdb.service.rpc.thrift.TCreateTimeseriesUsingSchemaTemplateReq; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; @@ -1573,7 +1575,12 @@ public TSExecuteStatementResp executePreparedStatement(TSExecutePreparedReq req) String.format("Prepared statement '%s' does not exist", statementName))); } - List parameters = deserializeParameters(req.getParameters()); + List rawParams = + PreparedParameterSerializer.deserialize(ByteBuffer.wrap(req.getParameters())); + List parameters = new ArrayList<>(rawParams.size()); + for (DeserializedParam param : rawParams) { + parameters.add(convertToLiteral(param)); + } org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement = preparedInfo.getSql(); @@ -1589,7 +1596,7 @@ public TSExecuteStatementResp executePreparedStatement(TSExecutePreparedReq req) } // Request query ID - queryId = SESSION_MANAGER.requestQueryId(clientSession, null); + queryId = SESSION_MANAGER.requestQueryId(clientSession, req.getStatementId()); // Execute using Coordinator with external parameters long timeout = req.isSetTimeout() ? req.getTimeout() : config.getQueryTimeoutThreshold(); @@ -1670,45 +1677,31 @@ public TSStatus deallocatePreparedStatement(TSDeallocatePreparedReq req) { } } - private List deserializeParameters(List params) { - List literals = new ArrayList<>(); - for (ByteBuffer buf : params) { - buf.rewind(); - byte type = buf.get(); - switch (type) { - case 0x00: // Null - literals.add(new NullLiteral()); - break; - case 0x01: // Boolean - boolean boolVal = buf.get() != 0; - literals.add(new BooleanLiteral(boolVal ? "true" : "false")); - break; - case 0x02: // Long - long longVal = buf.getLong(); - literals.add(new LongLiteral(String.valueOf(longVal))); - break; - case 0x03: // Double - double doubleVal = buf.getDouble(); - literals.add(new DoubleLiteral(doubleVal)); - break; - case 0x04: // String - int strLen = buf.getInt(); - byte[] strBytes = new byte[strLen]; - buf.get(strBytes); - literals.add( - new StringLiteral(new String(strBytes, java.nio.charset.StandardCharsets.UTF_8))); - break; - case 0x05: // Binary - int binLen = buf.getInt(); - byte[] binBytes = new byte[binLen]; - buf.get(binBytes); - literals.add(new BinaryLiteral(binBytes)); - break; - default: - throw new IllegalArgumentException("Unknown parameter type: " + type); - } + /** Convert a deserialized parameter to the corresponding Literal type for AST. */ + private Literal convertToLiteral(DeserializedParam param) { + if (param.isNull()) { + return new NullLiteral(); + } + + switch (param.type) { + case BOOLEAN: + return new BooleanLiteral((Boolean) param.value ? "true" : "false"); + case INT32: + return new LongLiteral(String.valueOf((Integer) param.value)); + case INT64: + return new LongLiteral(String.valueOf((Long) param.value)); + case FLOAT: + return new DoubleLiteral((Float) param.value); + case DOUBLE: + return new DoubleLiteral((Double) param.value); + case TEXT: + case STRING: + return new StringLiteral((String) param.value); + case BLOB: + return new BinaryLiteral((byte[]) param.value); + default: + throw new IllegalArgumentException("Unknown parameter type: " + param.type); } - return literals; } private final SelectResult setResultForPrepared = diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift index e751b9b4e713..f86be425a1a3 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift @@ -184,17 +184,18 @@ struct TSPrepareResp { // Executes a prepared statement with bound parameters struct TSExecutePreparedReq { 1: required i64 sessionId - 2: required string statementName // Name of the prepared statement - 3: required list parameters // Serialized parameter values + 2: required string statementName + 3: required binary parameters 4: optional i32 fetchSize 5: optional i64 timeout + 6: required i64 statementId } // PreparedStatement - DEALLOCATE // Releases a prepared statement and its resources struct TSDeallocatePreparedReq { 1: required i64 sessionId - 2: required string statementName // Name of the prepared statement to release + 2: required string statementName } struct TSFetchResultsReq{ From c7e2ec010d085e58acc4a80b71b530311b5cb5af Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Sun, 18 Jan 2026 22:47:33 +0800 Subject: [PATCH 4/8] Restore the original version of Coordinate --- .../db/queryengine/plan/Coordinator.java | 80 +++++++++++-------- 1 file changed, 45 insertions(+), 35 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index 87ae0c65539d..0007f5dc7974 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -40,6 +40,7 @@ import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; import org.apache.iotdb.db.queryengine.common.DataNodeEndPoints; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext.ExplainType; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; import org.apache.iotdb.db.queryengine.execution.QueryIdGenerator; @@ -65,9 +66,11 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations.PlanOptimizer; import org.apache.iotdb.db.queryengine.plan.relational.sql.ParameterExtractor; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AddColumn; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AlterColumnDataType; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AlterDB; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ClearCache; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CreateDB; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CreateExternalService; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CreateFunction; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CreateModel; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CreateTable; @@ -77,6 +80,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DescribeTable; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DropColumn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DropDB; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DropExternalService; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DropFunction; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DropModel; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DropTable; @@ -93,6 +97,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Parameter; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.PipeStatement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Prepare; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ReconstructRegion; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.RelationalAuthorStatement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.RemoveAINode; @@ -127,9 +132,12 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowTables; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowVariables; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowVersion; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StartExternalService; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StartRepairData; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StopExternalService; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.StopRepairData; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SubscriptionStatement; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.UnloadModel; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Use; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WrappedInsertStatement; @@ -394,6 +402,18 @@ private IQueryExecution createQueryExecutionForTreeModel( return new QueryExecution(treeModelPlanner, queryContext, executor); } + /** + * This method is specifically used following subquery: + * + *

1. When uncorrelated scalar subquery is handled + * (fetchUncorrelatedSubqueryResultForPredicate), we try to fold it and get constant value. Since + * CTE might be referenced, we need to add CTE materialization result into subquery's + * MPPQueryContext. + * + *

2. When CTE subquery is handled (fetchCteQueryResult), the main query, however, might be + * 'Explain' or 'Explain Analyze' statement. So we need to keep explain/explain analyze results + * along with CTE query dataset. + */ public ExecutionResult executeForTableModel( org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement, SqlParser sqlParser, @@ -402,38 +422,30 @@ public ExecutionResult executeForTableModel( SessionInfo session, String sql, Metadata metadata, + Map, Query> cteQueries, + ExplainType explainType, long timeOut, boolean userQuery) { - // Delegate to overloaded version with empty parameters - return executeForTableModel( - statement, - sqlParser, - clientSession, + return execution( queryId, session, sql, - metadata, - timeOut, userQuery, - Collections.emptyList()); + ((queryContext, startTime) -> { + queryContext.setInnerTriggeredQuery(true); + queryContext.setCteQueries(cteQueries); + queryContext.setExplainType(explainType); + return createQueryExecutionForTableModel( + statement, + sqlParser, + clientSession, + queryContext, + metadata, + timeOut > 0 ? timeOut : CONFIG.getQueryTimeoutThreshold(), + startTime); + })); } - /** - * Execute a table model statement with optional pre-bound parameters. Used by JDBC - * PreparedStatement to execute cached AST with serialized parameters. - * - * @param statement The AST to execute - * @param sqlParser SQL parser instance - * @param clientSession Current client session - * @param queryId Query ID - * @param session Session info - * @param sql SQL string for logging - * @param metadata Metadata instance - * @param timeOut Query timeout - * @param userQuery Whether this is a user query - * @param externalParameters List of Literal parameters to bind (empty for normal execution) - * @return ExecutionResult containing execution status and query ID - */ public ExecutionResult executeForTableModel( org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement, SqlParser sqlParser, @@ -443,8 +455,7 @@ public ExecutionResult executeForTableModel( String sql, Metadata metadata, long timeOut, - boolean userQuery, - List externalParameters) { + boolean userQuery) { return execution( queryId, session, @@ -458,8 +469,7 @@ public ExecutionResult executeForTableModel( queryContext, metadata, timeOut > 0 ? timeOut : CONFIG.getQueryTimeoutThreshold(), - startTime, - externalParameters))); + startTime))); } public ExecutionResult executeForTableModel( @@ -523,8 +533,7 @@ private IQueryExecution createQueryExecutionForTableModel( final MPPQueryContext queryContext, final Metadata metadata, final long timeOut, - final long startTime, - final List externalParameters) { + final long startTime) { queryContext.setTimeOut(timeOut); queryContext.setStartTime(startTime); if (statement instanceof DropDB @@ -536,6 +545,7 @@ private IQueryExecution createQueryExecutionForTableModel( || statement instanceof DescribeTable || statement instanceof ShowTables || statement instanceof AddColumn + || statement instanceof AlterColumnDataType || statement instanceof SetProperties || statement instanceof DropColumn || statement instanceof DropTable @@ -575,6 +585,10 @@ private IQueryExecution createQueryExecutionForTableModel( || statement instanceof CreateFunction || statement instanceof DropFunction || statement instanceof ShowFunctions + || statement instanceof CreateExternalService + || statement instanceof StartExternalService + || statement instanceof StopExternalService + || statement instanceof DropExternalService || statement instanceof RelationalAuthorStatement || statement instanceof MigrateRegion || statement instanceof ReconstructRegion @@ -604,11 +618,7 @@ private IQueryExecution createQueryExecutionForTableModel( List parameters = Collections.emptyList(); Map, Expression> parameterLookup = Collections.emptyMap(); - // Handle external parameters from JDBC PreparedStatement (highest priority) - if (externalParameters != null && !externalParameters.isEmpty()) { - parameterLookup = ParameterExtractor.bindParameters(statement, externalParameters); - parameters = new ArrayList<>(externalParameters); - } else if (statement instanceof Execute) { + if (statement instanceof Execute) { Execute executeStatement = (Execute) statement; String statementName = executeStatement.getStatementName().getValue(); From 45d563ff082df50e06b4f2527e47f9daca88a93d Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Sun, 18 Jan 2026 23:00:09 +0800 Subject: [PATCH 5/8] Refactor executePreparedStatement to reuse Coordinator's Execute AST handling --- .../thrift/impl/ClientRPCServiceImpl.java | 32 +++++-------------- 1 file changed, 8 insertions(+), 24 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index efbd972f645e..ecf89bdada4d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -95,6 +95,8 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BinaryLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.BooleanLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.DoubleLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Execute; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.NullLiteral; @@ -1567,14 +1569,7 @@ public TSExecuteStatementResp executePreparedStatement(TSExecutePreparedReq req) try { String statementName = req.getStatementName(); - PreparedStatementInfo preparedInfo = clientSession.getPreparedStatement(statementName); - if (preparedInfo == null) { - return RpcUtils.getTSExecuteStatementResp( - RpcUtils.getStatus( - TSStatusCode.EXECUTE_STATEMENT_ERROR, - String.format("Prepared statement '%s' does not exist", statementName))); - } - + // Deserialize parameters and convert to Literal list List rawParams = PreparedParameterSerializer.deserialize(ByteBuffer.wrap(req.getParameters())); List parameters = new ArrayList<>(rawParams.size()); @@ -1582,27 +1577,17 @@ public TSExecuteStatementResp executePreparedStatement(TSExecutePreparedReq req) parameters.add(convertToLiteral(param)); } - org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement = - preparedInfo.getSql(); - - int expectedCount = ParameterExtractor.getParameterCount(statement); - if (parameters.size() != expectedCount) { - return RpcUtils.getTSExecuteStatementResp( - RpcUtils.getStatus( - TSStatusCode.EXECUTE_STATEMENT_ERROR, - String.format( - "Parameter count mismatch: expected %d, got %d", - expectedCount, parameters.size()))); - } + // Construct Execute AST node, reuse Coordinator's existing Execute handling logic + Execute executeStatement = new Execute(new Identifier(statementName), parameters); // Request query ID queryId = SESSION_MANAGER.requestQueryId(clientSession, req.getStatementId()); - // Execute using Coordinator with external parameters + // Execute using Coordinator (Coordinator internally handles Execute statement) long timeout = req.isSetTimeout() ? req.getTimeout() : config.getQueryTimeoutThreshold(); ExecutionResult result = COORDINATOR.executeForTableModel( - statement, + executeStatement, relationSqlParser, clientSession, queryId, @@ -1610,8 +1595,7 @@ public TSExecuteStatementResp executePreparedStatement(TSExecutePreparedReq req) "EXECUTE " + statementName, metadata, timeout, - true, - parameters); + true); if (result.status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode() && result.status.code != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { From 2eb4569d80d6d97379e27c97729064df31547beb Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Sun, 18 Jan 2026 23:18:11 +0800 Subject: [PATCH 6/8] refactor(prepared-statement): extract helper for register/unregister logic --- .../thrift/impl/ClientRPCServiceImpl.java | 34 ++---- .../config/session/DeallocateTask.java | 19 +--- .../execution/config/session/PrepareTask.java | 27 +---- .../session/PreparedStatementHelper.java | 104 ++++++++++++++++++ 4 files changed, 121 insertions(+), 63 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PreparedStatementHelper.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index ecf89bdada4d..7b5c9810b71c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -53,7 +53,6 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.protocol.session.IClientSession; -import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.protocol.thrift.OperationType; import org.apache.iotdb.db.queryengine.common.SessionInfo; @@ -76,7 +75,7 @@ import org.apache.iotdb.db.queryengine.plan.analyze.schema.ISchemaFetcher; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; import org.apache.iotdb.db.queryengine.plan.execution.IQueryExecution; -import org.apache.iotdb.db.queryengine.plan.execution.config.session.PreparedStatementMemoryManager; +import org.apache.iotdb.db.queryengine.plan.execution.config.session.PreparedStatementHelper; import org.apache.iotdb.db.queryengine.plan.parser.ASTVisitor; import org.apache.iotdb.db.queryengine.plan.parser.StatementGenerator; import org.apache.iotdb.db.queryengine.plan.planner.LocalExecutionPlanner; @@ -1519,13 +1518,7 @@ public TSPrepareResp prepareStatement(TSPrepareReq req) { String sql = req.getSql(); String statementName = req.getStatementName(); - if (clientSession.getPreparedStatement(statementName) != null) { - return new TSPrepareResp( - RpcUtils.getStatus( - TSStatusCode.EXECUTE_STATEMENT_ERROR, - String.format("Prepared statement '%s' already exists", statementName))); - } - + // Parse SQL to get Statement AST org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement statement = relationSqlParser.createStatement(sql, clientSession.getZoneId(), clientSession); @@ -1534,15 +1527,11 @@ public TSPrepareResp prepareStatement(TSPrepareReq req) { RpcUtils.getStatus(TSStatusCode.SQL_PARSE_ERROR, "Failed to parse SQL: " + sql)); } + // Get parameter count before registering int parameterCount = ParameterExtractor.getParameterCount(statement); - long memorySizeInBytes = statement.ramBytesUsed(); - - PreparedStatementMemoryManager.getInstance().allocate(statementName, memorySizeInBytes); - - PreparedStatementInfo info = - new PreparedStatementInfo(statementName, statement, memorySizeInBytes); - clientSession.addPreparedStatement(statementName, info); + // Register the prepared statement using helper + PreparedStatementHelper.register(clientSession, statementName, statement); TSPrepareResp resp = new TSPrepareResp(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS)); resp.setParameterCount(parameterCount); @@ -1643,17 +1632,8 @@ public TSStatus deallocatePreparedStatement(TSDeallocatePreparedReq req) { } try { - String statementName = req.getStatementName(); - - PreparedStatementInfo removedInfo = clientSession.removePreparedStatement(statementName); - if (removedInfo == null) { - return RpcUtils.getStatus( - TSStatusCode.EXECUTE_STATEMENT_ERROR, - String.format("Prepared statement '%s' does not exist", statementName)); - } - - PreparedStatementMemoryManager.getInstance().release(removedInfo.getMemorySizeInBytes()); - + // Unregister the prepared statement using helper + PreparedStatementHelper.unregister(clientSession, req.getStatementName()); return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS); } catch (Exception e) { return onQueryException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/DeallocateTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/DeallocateTask.java index 6f5f3f484615..973f7cfa1f2c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/DeallocateTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/DeallocateTask.java @@ -19,9 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.execution.config.session; -import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.protocol.session.IClientSession; -import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; import org.apache.iotdb.db.queryengine.plan.execution.config.IConfigTask; @@ -54,19 +52,12 @@ public ListenableFuture execute(IConfigTaskExecutor configTask return future; } - // Remove the prepared statement - PreparedStatementInfo removedInfo = session.removePreparedStatement(statementName); - if (removedInfo == null) { - future.setException( - new SemanticException( - String.format("Prepared statement '%s' does not exist", statementName))); - return future; + try { + PreparedStatementHelper.unregister(session, statementName); + future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); + } catch (Exception e) { + future.setException(e); } - - // Release the memory allocated for this PreparedStatement from the shared MemoryBlock - PreparedStatementMemoryManager.getInstance().release(removedInfo.getMemorySizeInBytes()); - - future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); return future; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PrepareTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PrepareTask.java index 62c59d5bf785..c83808f4ceca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PrepareTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PrepareTask.java @@ -19,9 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.execution.config.session; -import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.protocol.session.IClientSession; -import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.plan.execution.config.ConfigTaskResult; import org.apache.iotdb.db.queryengine.plan.execution.config.IConfigTask; @@ -58,27 +56,12 @@ public ListenableFuture execute(IConfigTaskExecutor configTask return future; } - // Check if prepared statement with the same name already exists - PreparedStatementInfo existingInfo = session.getPreparedStatement(statementName); - if (existingInfo != null) { - future.setException( - new SemanticException( - String.format("Prepared statement '%s' already exists.", statementName))); - return future; + try { + PreparedStatementHelper.register(session, statementName, sql); + future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); + } catch (Exception e) { + future.setException(e); } - - // Estimate memory size of the AST - long memorySizeInBytes = sql == null ? 0L : sql.ramBytesUsed(); - - // Allocate memory from CoordinatorMemoryManager - // This memory is shared across all sessions using a single MemoryBlock - PreparedStatementMemoryManager.getInstance().allocate(statementName, memorySizeInBytes); - - // Create and store the prepared statement info (AST is cached) - PreparedStatementInfo info = new PreparedStatementInfo(statementName, sql, memorySizeInBytes); - session.addPreparedStatement(statementName, info); - - future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); return future; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PreparedStatementHelper.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PreparedStatementHelper.java new file mode 100644 index 000000000000..d6f890de6934 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/session/PreparedStatementHelper.java @@ -0,0 +1,104 @@ +/* + * 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.iotdb.db.queryengine.plan.execution.config.session; + +import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.protocol.session.IClientSession; +import org.apache.iotdb.db.protocol.session.PreparedStatementInfo; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement; + +/** + * Helper class for managing prepared statement registration and unregistration. Provides common + * logic shared between RPC methods and ConfigTask implementations. + */ +public class PreparedStatementHelper { + + private PreparedStatementHelper() { + // Utility class + } + + /** + * Registers a prepared statement in the session. + * + *

This method performs the following operations: + * + *

    + *
  1. Checks if a prepared statement with the same name already exists + *
  2. Calculates memory size of the AST + *
  3. Allocates memory from PreparedStatementMemoryManager + *
  4. Creates and stores PreparedStatementInfo in the session + *
+ * + * @param session the client session + * @param statementName the name of the prepared statement + * @param sql the parsed SQL statement (AST) + * @return the created PreparedStatementInfo + * @throws SemanticException if a prepared statement with the same name already exists + */ + public static PreparedStatementInfo register( + IClientSession session, String statementName, Statement sql) { + // Check if prepared statement with the same name already exists + if (session.getPreparedStatement(statementName) != null) { + throw new SemanticException( + String.format("Prepared statement '%s' already exists", statementName)); + } + + // Calculate memory size of the AST + long memorySizeInBytes = sql == null ? 0L : sql.ramBytesUsed(); + + // Allocate memory from PreparedStatementMemoryManager + PreparedStatementMemoryManager.getInstance().allocate(statementName, memorySizeInBytes); + + // Create and store PreparedStatementInfo + PreparedStatementInfo info = new PreparedStatementInfo(statementName, sql, memorySizeInBytes); + session.addPreparedStatement(statementName, info); + + return info; + } + + /** + * Unregisters a prepared statement from the session. + * + *

This method performs the following operations: + * + *

    + *
  1. Removes the prepared statement from the session + *
  2. Releases the allocated memory + *
+ * + * @param session the client session + * @param statementName the name of the prepared statement to remove + * @return the removed PreparedStatementInfo + * @throws SemanticException if the prepared statement does not exist + */ + public static PreparedStatementInfo unregister(IClientSession session, String statementName) { + // Remove the prepared statement + PreparedStatementInfo removedInfo = session.removePreparedStatement(statementName); + if (removedInfo == null) { + throw new SemanticException( + String.format("Prepared statement '%s' does not exist", statementName)); + } + + // Release the allocated memory + PreparedStatementMemoryManager.getInstance().release(removedInfo.getMemorySizeInBytes()); + + return removedInfo; + } +} From 031fcd4c71b113373977e663d3691add1bfe7488 Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Sun, 18 Jan 2026 23:22:44 +0800 Subject: [PATCH 7/8] use StatementId instead of UUID for prepared statement name --- .../apache/iotdb/jdbc/IoTDBTablePreparedStatement.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java index 4bcf008f29bf..3a1f27f68078 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java @@ -71,7 +71,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.UUID; public class IoTDBTablePreparedStatement extends IoTDBStatement implements PreparedStatement { @@ -119,9 +118,7 @@ public class IoTDBTablePreparedStatement extends IoTDBStatement implements Prepa for (int i = 0; i < parameterCount; i++) { parameterTypes[i] = Types.NULL; } - } catch (TException e) { - throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); - } catch (StatementExecutionException e) { + } catch (TException | StatementExecutionException e) { throw new SQLException("Failed to prepare statement: " + e.getMessage(), e); } } @@ -134,7 +131,8 @@ public class IoTDBTablePreparedStatement extends IoTDBStatement implements Prepa } private String generateStatementName() { - return "jdbc_ps_" + UUID.randomUUID().toString().replace("-", ""); + // StatementId is unique across all sessions in one IoTDB instance + return "jdbc_ps_" + getStmtId(); } @Override From 7b3ec212f8a9b1e720324de34876ba11deb56f4a Mon Sep 17 00:00:00 2001 From: Young-Leo <562593859@qq.com> Date: Mon, 19 Jan 2026 15:04:28 +0800 Subject: [PATCH 8/8] Add unit tests for PreparedParameterSerializer --- ...ava => IoTDBTablePreparedStatementIT.java} | 2 +- .../jdbc/IoTDBTablePreparedStatement.java | 4 +- .../rpc/stmt/PreparedParameterSerializer.java | 5 +- .../stmt/PreparedParameterSerializerTest.java | 127 ++++++++++++++++++ .../db/protocol/thrift/OperationType.java | 5 +- .../thrift/impl/ClientRPCServiceImpl.java | 10 +- 6 files changed, 144 insertions(+), 9 deletions(-) rename integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/{IoTDBPreparedStatementIT.java => IoTDBTablePreparedStatementIT.java} (99%) create mode 100644 iotdb-client/service-rpc/src/test/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializerTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBPreparedStatementIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBTablePreparedStatementIT.java similarity index 99% rename from integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBPreparedStatementIT.java rename to integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBTablePreparedStatementIT.java index f06d46201aff..c7c5902e8e2d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBPreparedStatementIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/db/it/IoTDBTablePreparedStatementIT.java @@ -45,7 +45,7 @@ @RunWith(IoTDBTestRunner.class) @Category({TableLocalStandaloneIT.class, TableClusterIT.class}) -public class IoTDBPreparedStatementIT { +public class IoTDBTablePreparedStatementIT { private static final String DATABASE_NAME = "test"; private static final String[] sqls = new String[] { diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java index 3a1f27f68078..bebaf2455b4c 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBTablePreparedStatement.java @@ -192,9 +192,7 @@ private TSExecuteStatementResp executeInternal() throws SQLException { TSExecuteStatementResp resp = client.executePreparedStatement(req); RpcUtils.verifySuccess(resp.getStatus()); return resp; - } catch (TException e) { - throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); - } catch (StatementExecutionException e) { + } catch (TException | StatementExecutionException e) { throw new SQLException("Failed to execute prepared statement: " + e.getMessage(), e); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java index 81d25bc773b7..3b668bffb896 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializer.java @@ -39,7 +39,6 @@ */ public class PreparedParameterSerializer { - /** Deserialized parameter holding type and value. */ public static class DeserializedParam { public final TSDataType type; public final Object value; @@ -156,6 +155,10 @@ public static List deserialize(ByteBuffer buffer) { buffer.rewind(); int count = buffer.getInt(); + if (count < 0 || count > buffer.remaining()) { + throw new IllegalArgumentException("Invalid parameter count: " + count); + } + List result = new ArrayList<>(count); for (int i = 0; i < count; i++) { diff --git a/iotdb-client/service-rpc/src/test/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializerTest.java b/iotdb-client/service-rpc/src/test/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializerTest.java new file mode 100644 index 000000000000..10e3cd4b6788 --- /dev/null +++ b/iotdb-client/service-rpc/src/test/java/org/apache/iotdb/rpc/stmt/PreparedParameterSerializerTest.java @@ -0,0 +1,127 @@ +/* + * 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.iotdb.rpc.stmt; + +import org.apache.iotdb.rpc.stmt.PreparedParameterSerializer.DeserializedParam; + +import org.apache.tsfile.enums.TSDataType; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.sql.Types; +import java.util.List; + +import static org.apache.iotdb.rpc.stmt.PreparedParameterSerializer.deserialize; +import static org.apache.iotdb.rpc.stmt.PreparedParameterSerializer.serialize; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** Unit tests for {@link PreparedParameterSerializer}. */ +public class PreparedParameterSerializerTest { + + @Test + public void testEmptyParameterList() { + ByteBuffer buffer = serialize(new Object[0], new int[0], 0); + List result = deserialize(buffer); + + assertNotNull(result); + assertTrue(result.isEmpty()); + } + + @Test + public void testNullAndEmptyBuffer() { + assertTrue(deserialize(null).isEmpty()); + assertTrue(deserialize(ByteBuffer.allocate(0)).isEmpty()); + } + + @Test + public void testNullValue() { + ByteBuffer buffer = serialize(new Object[] {null}, new int[] {Types.VARCHAR}, 1); + List result = deserialize(buffer); + + assertEquals(1, result.size()); + assertTrue(result.get(0).isNull()); + } + + @Test + public void testAllDataTypes() { + Object[] values = {true, 42, 123456789L, 3.14f, 2.71828, "hello", new byte[] {1, 2, 3}}; + int[] types = { + Types.BOOLEAN, + Types.INTEGER, + Types.BIGINT, + Types.FLOAT, + Types.DOUBLE, + Types.VARCHAR, + Types.BINARY + }; + + ByteBuffer buffer = serialize(values, types, 7); + List result = deserialize(buffer); + + assertEquals(7, result.size()); + assertEquals(TSDataType.BOOLEAN, result.get(0).type); + assertEquals(true, result.get(0).value); + assertEquals(TSDataType.INT32, result.get(1).type); + assertEquals(42, result.get(1).value); + assertEquals(TSDataType.INT64, result.get(2).type); + assertEquals(123456789L, result.get(2).value); + assertEquals(TSDataType.FLOAT, result.get(3).type); + assertEquals(3.14f, (Float) result.get(3).value, 0.0001f); + assertEquals(TSDataType.DOUBLE, result.get(4).type); + assertEquals(2.71828, (Double) result.get(4).value, 0.00001); + assertEquals(TSDataType.STRING, result.get(5).type); + assertEquals("hello", result.get(5).value); + assertEquals(TSDataType.BLOB, result.get(6).type); + assertArrayEquals(new byte[] {1, 2, 3}, (byte[]) result.get(6).value); + } + + @Test + public void testUnicodeString() { + ByteBuffer buffer = serialize(new Object[] {"你好🌍"}, new int[] {Types.VARCHAR}, 1); + List result = deserialize(buffer); + + assertEquals("你好🌍", result.get(0).value); + } + + @Test + public void testMixedNullAndValues() { + Object[] values = {"hello", null, 42}; + int[] types = {Types.VARCHAR, Types.INTEGER, Types.INTEGER}; + + ByteBuffer buffer = serialize(values, types, 3); + List result = deserialize(buffer); + + assertEquals(3, result.size()); + assertEquals("hello", result.get(0).value); + assertTrue(result.get(1).isNull()); + assertEquals(42, result.get(2).value); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidParameterCount() { + ByteBuffer buffer = ByteBuffer.allocate(4); + buffer.putInt(-1); + buffer.flip(); + deserialize(buffer); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java index e461f0cc45d1..9c44de9f5fdc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java @@ -52,7 +52,10 @@ public enum OperationType { EXECUTE_NON_QUERY_PLAN("executeNonQueryPlan"), SELECT_INTO("selectInto"), QUERY_LATENCY("queryLatency"), - WRITE_AUDIT_LOG("writeAuditLog"); + WRITE_AUDIT_LOG("writeAuditLog"), + PREPARE_STATEMENT("prepareStatement"), + EXECUTE_PREPARED_STATEMENT("executePreparedStatement"), + DEALLOCATE_PREPARED_STATEMENT("deallocatePreparedStatement"); private final String name; OperationType(String name) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index 7b5c9810b71c..1a9ca6e8e62e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -1539,7 +1539,7 @@ public TSPrepareResp prepareStatement(TSPrepareReq req) { } catch (Exception e) { return new TSPrepareResp( onQueryException( - e, OperationType.EXECUTE_STATEMENT.getName(), TSStatusCode.INTERNAL_SERVER_ERROR)); + e, OperationType.PREPARE_STATEMENT.getName(), TSStatusCode.INTERNAL_SERVER_ERROR)); } } @@ -1614,7 +1614,9 @@ public TSExecuteStatementResp executePreparedStatement(TSExecutePreparedReq req) t = e; return RpcUtils.getTSExecuteStatementResp( onQueryException( - e, OperationType.EXECUTE_STATEMENT.getName(), TSStatusCode.INTERNAL_SERVER_ERROR)); + e, + OperationType.EXECUTE_PREPARED_STATEMENT.getName(), + TSStatusCode.INTERNAL_SERVER_ERROR)); } finally { long currentOperationCost = System.nanoTime() - startTime; if (finished) { @@ -1637,7 +1639,9 @@ public TSStatus deallocatePreparedStatement(TSDeallocatePreparedReq req) { return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS); } catch (Exception e) { return onQueryException( - e, OperationType.EXECUTE_STATEMENT.getName(), TSStatusCode.INTERNAL_SERVER_ERROR); + e, + OperationType.DEALLOCATE_PREPARED_STATEMENT.getName(), + TSStatusCode.INTERNAL_SERVER_ERROR); } }