Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 9 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -26,3 +26,12 @@ out/
.antlr/
log/
target/

# Generated files
src/main/java/ru/yandex/clickhouse/jdbc/parser/*CharStream.java
src/main/java/ru/yandex/clickhouse/jdbc/parser/ClickHouseSqlParser*.java
src/main/java/ru/yandex/clickhouse/jdbc/parser/Token*.java
src/main/java/ru/yandex/clickhouse/jdbc/parser/ParseException.java

# Shell scripts
*.sh
22 changes: 22 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@
<deploy-plugin.version>3.0.0-M1</deploy-plugin.version>
<staging-plugin.version>1.6.8</staging-plugin.version>
<gpg-plugin.version>1.6</gpg-plugin.version>
<javacc-plugin.version>4.1.4</javacc-plugin.version>
<compiler-plugin.version>3.8.1</compiler-plugin.version>
<source-plugin.version>3.2.1</source-plugin.version>
<jar-plugin.version>3.2.0</jar-plugin.version>
Expand Down Expand Up @@ -288,6 +289,27 @@
<useSystemClassLoader>false</useSystemClassLoader>
</configuration>
</plugin>
<plugin>
<groupId>com.helger.maven</groupId>
<artifactId>ph-javacc-maven-plugin</artifactId>
<version>${javacc-plugin.version}</version>
<executions>
<execution>
<id>jjc</id>
<phase>generate-sources</phase>
<goals>
<goal>javacc</goal>
</goals>
<configuration>
<jdkVersion>${jdk.version}</jdkVersion>
<javadocFriendlyComments>true</javadocFriendlyComments>
<packageName>ru.yandex.clickhouse.jdbc.parser</packageName>
<sourceDirectory>src/main/javacc</sourceDirectory>
<outputDirectory>src/main/java</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,14 +36,15 @@
import org.apache.http.entity.AbstractHttpEntity;
import org.apache.http.impl.client.CloseableHttpClient;

import ru.yandex.clickhouse.jdbc.parser.ClickHouseSqlStatement;
import ru.yandex.clickhouse.jdbc.parser.StatementType;
import ru.yandex.clickhouse.response.ClickHouseResponse;
import ru.yandex.clickhouse.settings.ClickHouseProperties;
import ru.yandex.clickhouse.settings.ClickHouseQueryParam;
import ru.yandex.clickhouse.util.ClickHouseArrayUtil;
import ru.yandex.clickhouse.util.ClickHouseValueFormatter;
import ru.yandex.clickhouse.util.guava.StreamUtils;


public class ClickHousePreparedStatementImpl extends ClickHouseStatementImpl implements ClickHousePreparedStatement {

static final String PARAM_MARKER = "?";
Expand All @@ -65,8 +66,11 @@ public ClickHousePreparedStatementImpl(CloseableHttpClient client,
TimeZone serverTimeZone, int resultSetType) throws SQLException
{
super(client, connection, properties, resultSetType);
parseSingleStatement(sql);

this.sql = sql;
PreparedStatementParser parser = PreparedStatementParser.parse(sql);
PreparedStatementParser parser = PreparedStatementParser.parse(sql,
parsedStmt.getEndPosition(ClickHouseSqlStatement.KEYWORD_VALUES));
this.parameterList = parser.getParameters();
this.insertBatchMode = parser.isValuesMode();
this.sqlParts = parser.getParts();
Expand Down Expand Up @@ -347,14 +351,22 @@ public int[] executeBatch() throws SQLException {

@Override
public int[] executeBatch(Map<ClickHouseQueryParam, String> additionalDBParams) throws SQLException {
Matcher matcher = VALUES.matcher(sql);
if (!matcher.find()) {
int valuePosition = -1;
if (parsedStmt.getStatementType() == StatementType.INSERT && parsedStmt.hasValues()) {
valuePosition = parsedStmt.getStartPosition(ClickHouseSqlStatement.KEYWORD_VALUES);
} else {
Matcher matcher = VALUES.matcher(sql);
if (matcher.find()) {
valuePosition = matcher.start();
}
}

if (valuePosition < 0) {
throw new SQLSyntaxErrorException(
"Query must be like 'INSERT INTO [db.]table [(c1, c2, c3)] VALUES (?, ?, ?)'. " +
"Got: " + sql
);
}
int valuePosition = matcher.start();
String insertSql = sql.substring(0, valuePosition);
BatchHttpEntity entity = new BatchHttpEntity(batchRows);
sendStream(entity, insertSql, additionalDBParams);
Expand Down Expand Up @@ -429,7 +441,8 @@ public ResultSetMetaData getMetaData() throws SQLException {
if (currentResult != null) {
return currentResult.getMetaData();
}
if (!isSelect(sql)) {

if (!parsedStmt.isQuery() || (!parsedStmt.isRecognized() && !isSelect(sql))) {
return null;
}
ResultSet myRs = executeQuery(Collections.singletonMap(
Expand Down
120 changes: 107 additions & 13 deletions src/main/java/ru/yandex/clickhouse/ClickHouseStatementImpl.java
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,9 @@
import java.sql.SQLWarning;
import java.util.ArrayList;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.TimeZone;
import java.util.UUID;
Expand All @@ -38,6 +40,9 @@
import ru.yandex.clickhouse.domain.ClickHouseFormat;
import ru.yandex.clickhouse.except.ClickHouseException;
import ru.yandex.clickhouse.except.ClickHouseExceptionSpecifier;
import ru.yandex.clickhouse.jdbc.parser.ClickHouseSqlParser;
import ru.yandex.clickhouse.jdbc.parser.ClickHouseSqlStatement;
import ru.yandex.clickhouse.jdbc.parser.StatementType;
import ru.yandex.clickhouse.response.ClickHouseLZ4Stream;
import ru.yandex.clickhouse.response.ClickHouseResponse;
import ru.yandex.clickhouse.response.ClickHouseResponseSummary;
Expand Down Expand Up @@ -84,16 +89,56 @@ public class ClickHouseStatementImpl extends ConfigurableApi<ClickHouseStatement

private volatile String queryId;

protected ClickHouseSqlStatement parsedStmt;

/**
* Current database name may be changed by {@link java.sql.Connection#setCatalog(String)}
* between creation of this object and query execution, but javadoc does not allow
* {@code setCatalog} influence on already created statements.
*/
private final String initialDatabase;

@Deprecated
private static final String[] selectKeywords = new String[]{"SELECT", "WITH", "SHOW", "DESC", "EXISTS", "EXPLAIN"};
@Deprecated
private static final String databaseKeyword = "CREATE DATABASE";

@Deprecated
protected void parseSingleStatement(String sql) throws SQLException {
this.parsedStmt = null;
ClickHouseSqlStatement[] stmts = ClickHouseSqlParser.parse(sql, properties);

if (stmts.length == 1) {
this.parsedStmt = stmts[0];
} else {
this.parsedStmt = new ClickHouseSqlStatement(sql, StatementType.UNKNOWN);
// throw new SQLException("Multiple statements are not supported.");
}

if (this.parsedStmt.isIdemponent()) {
httpContext.setAttribute("is_idempotent", Boolean.TRUE);
} else {
httpContext.removeAttribute("is_idempotent");
}
}

@Deprecated
private void parseSingleStatement(String sql, ClickHouseFormat preferredFormat) throws SQLException {
parseSingleStatement(sql);

if (parsedStmt.isQuery() && !parsedStmt.hasFormat()) {
String format = preferredFormat.name();
Map<String, Integer> positions = new HashMap<>();
positions.putAll(parsedStmt.getPositions());
positions.put(ClickHouseSqlStatement.KEYWORD_FORMAT, sql.length());

sql = new StringBuilder(parsedStmt.getSQL()).append("\nFORMAT ").append(format).append(';')
.toString();
parsedStmt = new ClickHouseSqlStatement(sql, parsedStmt.getStatementType(),
parsedStmt.getCluster(), parsedStmt.getDatabase(), parsedStmt.getTable(),
format, parsedStmt.getOutfile(), parsedStmt.getParameters(), positions);
}
}

public ClickHouseStatementImpl(CloseableHttpClient client, ClickHouseConnection connection,
ClickHouseProperties properties, int resultSetType) {
Expand Down Expand Up @@ -135,16 +180,29 @@ public ResultSet executeQuery(String sql,
}
additionalDBParams.put(ClickHouseQueryParam.EXTREMES, "0");

InputStream is = getInputStream(sql, additionalDBParams, externalData, additionalRequestParams);
parseSingleStatement(sql, ClickHouseFormat.TabSeparatedWithNamesAndTypes);
if (!parsedStmt.isRecognized() && isSelect(sql)) {
Map<String, Integer> positions = new HashMap<>();
String dbName = extractDBName(sql);
String tableName = extractTableName(sql);
if (extractWithTotals(sql)) {
positions.put(ClickHouseSqlStatement.KEYWORD_TOTALS, 1);
}
parsedStmt = new ClickHouseSqlStatement(sql, StatementType.SELECT,
null, dbName, tableName, null, null, null, positions);
// httpContext.setAttribute("is_idempotent", Boolean.TRUE);
}

InputStream is = getInputStream(sql, additionalDBParams, externalData, additionalRequestParams);

try {
if (isSelect(sql)) {
if (parsedStmt.isQuery()) {
currentUpdateCount = -1;
currentResult = createResultSet(properties.isCompress()
? new ClickHouseLZ4Stream(is) : is, properties.getBufferSize(),
extractDBName(sql),
extractTableName(sql),
extractWithTotals(sql),
parsedStmt.getDatabaseOrDefault(properties.getDatabase()),
parsedStmt.getTable(),
parsedStmt.hasWithTotals(),
this,
getConnection().getTimeZone(),
properties
Expand Down Expand Up @@ -176,8 +234,15 @@ public ClickHouseResponse executeQueryClickhouseResponse(String sql, Map<ClickHo
public ClickHouseResponse executeQueryClickhouseResponse(String sql,
Map<ClickHouseQueryParam, String> additionalDBParams,
Map<String, String> additionalRequestParams) throws SQLException {
parseSingleStatement(sql, ClickHouseFormat.JSONCompact);
if (parsedStmt.isRecognized()) {
sql = parsedStmt.getSQL();
} else {
sql = addFormatIfAbsent(sql, ClickHouseFormat.JSONCompact);
}

InputStream is = getInputStream(
addFormatIfAbsent(sql, ClickHouseFormat.JSONCompact),
sql,
additionalDBParams,
null,
additionalRequestParams
Expand Down Expand Up @@ -206,14 +271,27 @@ public ClickHouseRowBinaryInputStream executeQueryClickhouseRowBinaryStream(Stri

@Override
public ClickHouseRowBinaryInputStream executeQueryClickhouseRowBinaryStream(String sql, Map<ClickHouseQueryParam, String> additionalDBParams, Map<String, String> additionalRequestParams) throws SQLException {
parseSingleStatement(sql, ClickHouseFormat.RowBinary);
if (parsedStmt.isRecognized()) {
sql = parsedStmt.getSQL();
} else {
sql = addFormatIfAbsent(sql, ClickHouseFormat.RowBinary);
if (isSelect(sql)) {
parsedStmt = new ClickHouseSqlStatement(sql, StatementType.SELECT);
// httpContext.setAttribute("is_idempotent", Boolean.TRUE);
} else {
parsedStmt = new ClickHouseSqlStatement(sql, StatementType.UNKNOWN);
}
}

InputStream is = getInputStream(
addFormatIfAbsent(sql, ClickHouseFormat.RowBinary),
sql,
additionalDBParams,
null,
additionalRequestParams
);
try {
if (isSelect(sql)) {
if (parsedStmt.isQuery()) {
currentUpdateCount = -1;
currentRowBinaryResult = new ClickHouseRowBinaryInputStream(properties.isCompress()
? new ClickHouseLZ4Stream(is) : is, getConnection().getTimeZone(), properties);
Expand All @@ -231,6 +309,8 @@ public ClickHouseRowBinaryInputStream executeQueryClickhouseRowBinaryStream(Stri

@Override
public int executeUpdate(String sql) throws SQLException {
parseSingleStatement(sql, ClickHouseFormat.TabSeparatedWithNamesAndTypes);

InputStream is = null;
try {
is = getInputStream(sql, null, null, null);
Expand All @@ -245,8 +325,7 @@ public int executeUpdate(String sql) throws SQLException {
@Override
public boolean execute(String sql) throws SQLException {
// currentResult is stored here. InputString and currentResult will be closed on this.close()
executeQuery(sql);
return isSelect(sql);
return executeQuery(sql) != null;
}

@Override
Expand Down Expand Up @@ -471,6 +550,7 @@ public ClickHouseResponseSummary getResponseSummary() {
return currentSummary;
}

@Deprecated
static String clickhousifySql(String sql) {
return addFormatIfAbsent(sql, ClickHouseFormat.TabSeparatedWithNamesAndTypes);
}
Expand All @@ -479,6 +559,7 @@ static String clickhousifySql(String sql) {
* Adding FORMAT TabSeparatedWithNamesAndTypes if not added
* adds format only to select queries
*/
@Deprecated
private static String addFormatIfAbsent(final String sql, ClickHouseFormat format) {
String cleanSQL = sql.trim();
if (!isSelect(cleanSQL)) {
Expand All @@ -498,6 +579,7 @@ private static String addFormatIfAbsent(final String sql, ClickHouseFormat forma
return sb.toString();
}

@Deprecated
static boolean isSelect(String sql) {
for (int i = 0; i < sql.length(); i++) {
String nextTwo = sql.substring(i, Math.min(i + 2, sql.length()));
Expand All @@ -518,6 +600,7 @@ static boolean isSelect(String sql) {
return false;
}

@Deprecated
private String extractTableName(String sql) {
String s = extractDBAndTableName(sql);
if (s.contains(".")) {
Expand All @@ -527,6 +610,7 @@ private String extractTableName(String sql) {
}
}

@Deprecated
private String extractDBName(String sql) {
String s = extractDBAndTableName(sql);
if (s.contains(".")) {
Expand All @@ -536,6 +620,7 @@ private String extractDBName(String sql) {
}
}

@Deprecated
private String extractDBAndTableName(String sql) {
if (Utils.startsWithIgnoreCase(sql, "select")) {
String withoutStrings = Utils.retainUnquoted(sql, '\'');
Expand All @@ -558,10 +643,11 @@ private String extractDBAndTableName(String sql) {
return "system.unknown";
}

@Deprecated
private boolean extractWithTotals(String sql) {
if (Utils.startsWithIgnoreCase(sql, "select")) {
String withoutStrings = Utils.retainUnquoted(sql, '\'');
return withoutStrings.toLowerCase().contains(" with totals");
return withoutStrings.toLowerCase(Locale.ROOT).contains(" with totals");
}
return false;
}
Expand All @@ -572,15 +658,23 @@ private InputStream getInputStream(
List<ClickHouseExternalData> externalData,
Map<String, String> additionalRequestParams
) throws ClickHouseException {
sql = clickhousifySql(sql);
boolean ignoreDatabase = false;
if (parsedStmt.isRecognized()) {
sql = parsedStmt.getSQL();
// TODO consider more scenarios like drop, show etc.
ignoreDatabase = parsedStmt.getStatementType() == StatementType.CREATE
&& parsedStmt.containsKeyword(ClickHouseSqlStatement.KEYWORD_DATABASE);
} else {
sql = clickhousifySql(sql);
ignoreDatabase = sql.trim().regionMatches(true, 0, databaseKeyword, 0, databaseKeyword.length());
}
log.debug("Executing SQL: {}", sql);

additionalClickHouseDBParams = addQueryIdTo(
additionalClickHouseDBParams == null
? new EnumMap<ClickHouseQueryParam, String>(ClickHouseQueryParam.class)
: additionalClickHouseDBParams);

boolean ignoreDatabase = sql.trim().regionMatches(true, 0, databaseKeyword, 0, databaseKeyword.length());
URI uri;
if (externalData == null || externalData.isEmpty()) {
uri = buildRequestUri(
Expand Down
Loading