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
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ static <K, V> ClickHouseCache<K, V> create(int capacity, long expireSeconds, Fun
* @param <T> type of the cache
* @param clazz non-null class of the cache
* @return inner cache object
* @throws NullPointerException
* @throws NullPointerException when {@code clazz} is null
*/
<T> T unwrap(Class<T> clazz);
}
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,23 @@ public static boolean isNullOrBlank(CharSequence value) {
return true;
}

/**
* Checks if the given string is not {@code null}, empty or blank and throws a
* customized {@link IllegalArgumentException} if it is.
*
* @param value the string to check
* @param name name of the string
* @return the exact same string
* @throws IllegalArgumentException if the string is null, empty or blank
*/
public static String nonBlank(String value, String name) {
if (isNullOrBlank(value)) {
throw newException("%s cannot be null, empty or blank string", name);
}

return value;
}

/**
* Checks if the given string is neither {@code null} nor empty and throws a
* customized {@link IllegalArgumentException} if it is.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -38,10 +37,6 @@
* implementation properly in runtime.
*/
public interface ClickHouseClient extends AutoCloseable {
/**
* Empty stringified parameters.
*/
static final List<Map<String, String>> EMPTY_STRINGIFIED_PARAMETERS = Collections.emptyList();

/**
* Returns a builder for creating a new client.
Expand Down Expand Up @@ -74,6 +69,7 @@ static ExecutorService getExecutorService() {
* @param <T> return type of the task
* @param task non-null task
* @return future object to get result
* @throws CompletionException when failed to complete the task
*/
static <T> CompletableFuture<T> submit(Callable<T> task) {
try {
Expand Down Expand Up @@ -101,7 +97,10 @@ static <T> CompletableFuture<T> submit(Callable<T> task) {
* @param compression compression algorithm to use
* @param file output file
* @return future object to get result
* @throws IOException when failed to create the file or its parent directories
* @throws IllegalArgumentException if any of server, tableOrQuery, and output
* is null
* @throws IOException when failed to create the file or its parent
* directories
*/
static CompletableFuture<ClickHouseResponseSummary> dump(ClickHouseNode server, String tableOrQuery,
ClickHouseFormat format, ClickHouseCompression compression, String file) throws IOException {
Expand All @@ -120,6 +119,8 @@ static CompletableFuture<ClickHouseResponseSummary> dump(ClickHouseNode server,
* @param output output stream, which will be closed automatically at the
* end of the call
* @return future object to get result
* @throws IllegalArgumentException if any of server, tableOrQuery, and output
* is null
*/
static CompletableFuture<ClickHouseResponseSummary> dump(ClickHouseNode server, String tableOrQuery,
ClickHouseFormat format, ClickHouseCompression compression, OutputStream output) {
Expand Down Expand Up @@ -148,8 +149,11 @@ static CompletableFuture<ClickHouseResponseSummary> dump(ClickHouseNode server,
response.dump(output);
return response.getSummary();
}
} catch (InterruptedException | ExecutionException e) {
throw ClickHouseExceptionSpecifier.specify(e, theServer.getHost(), theServer.getPort());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw ClickHouseExceptionSpecifier.specify(e, theServer);
} catch (ExecutionException e) {
throw ClickHouseExceptionSpecifier.handle(e, theServer);
} finally {
try {
output.close();
Expand All @@ -170,7 +174,8 @@ static CompletableFuture<ClickHouseResponseSummary> dump(ClickHouseNode server,
* @param compression compression algorithm to use
* @param file file to load
* @return future object to get result
* @throws FileNotFoundException when file not found
* @throws IllegalArgumentException if any of server, table, and input is null
* @throws FileNotFoundException when file not found
*/
static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server, String table,
ClickHouseFormat format, ClickHouseCompression compression, String file) throws FileNotFoundException {
Expand All @@ -187,6 +192,7 @@ static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server,
* @param compression compression algorithm to use
* @param writer non-null custom writer to generate data
* @return future object to get result
* @throws IllegalArgumentException if any of server, table, and writer is null
*/
static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server, String table,
ClickHouseFormat format, ClickHouseCompression compression, ClickHouseWriter writer) {
Expand Down Expand Up @@ -221,8 +227,11 @@ static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server,
try (ClickHouseResponse response = future.get()) {
return response.getSummary();
}
} catch (InterruptedException | ExecutionException e) {
throw ClickHouseExceptionSpecifier.specify(e, theServer.getHost(), theServer.getPort());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw ClickHouseExceptionSpecifier.specify(e, theServer);
} catch (ExecutionException e) {
throw ClickHouseExceptionSpecifier.handle(e, theServer);
} finally {
if (input != null) {
try {
Expand All @@ -246,6 +255,7 @@ static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server,
* @param input input stream, which will be closed automatically at the
* end of the call
* @return future object to get result
* @throws IllegalArgumentException if any of server, table, and input is null
*/
static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server, String table,
ClickHouseFormat format, ClickHouseCompression compression, InputStream input) {
Expand All @@ -264,8 +274,11 @@ static CompletableFuture<ClickHouseResponseSummary> load(ClickHouseNode server,
ClickHouseResponse response = client.connect(theServer).write().table(table)
.compression(theCompression).format(theFormat).data(input).execute().get()) {
return response.getSummary();
} catch (InterruptedException | ExecutionException e) {
throw ClickHouseExceptionSpecifier.specify(e, theServer.getHost(), theServer.getPort());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw ClickHouseExceptionSpecifier.specify(e, theServer);
} catch (ExecutionException e) {
throw ClickHouseExceptionSpecifier.handle(e, theServer);
} finally {
try {
input.close();
Expand Down Expand Up @@ -295,6 +308,7 @@ static ClickHouseClient newInstance(ClickHouseProtocol... preferredProtocols) {
* @param sql non-null SQL query
* @param more more SQL queries if any
* @return list of {@link ClickHouseResponseSummary} one for each execution
* @throws IllegalArgumentException if server or sql is null
*/
static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode server, String sql, String... more) {
if (server == null || sql == null) {
Expand Down Expand Up @@ -342,6 +356,7 @@ static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode se
* @param sql non-null SQL query
* @param params non-null stringified parameters
* @return list of {@link ClickHouseResponseSummary} one for each execution
* @throws IllegalArgumentException if any of server, sql, and params is null
*/
static CompletableFuture<ClickHouseResponseSummary> send(ClickHouseNode server, String sql,
Map<String, String> params) {
Expand Down Expand Up @@ -373,6 +388,8 @@ static CompletableFuture<ClickHouseResponseSummary> send(ClickHouseNode server,
* @param columns non-empty column list
* @param params non-empty raw parameters
* @return list of {@link ClickHouseResponseSummary} one for each execution
* @throws IllegalArgumentException if columns is null, empty or contains null
* column
*/
static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode server, String sql,
List<ClickHouseColumn> columns, Object[]... params) {
Expand All @@ -399,6 +416,8 @@ static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode se
* @param templates non-empty template objects to stringify parameters
* @param params non-empty raw parameters
* @return list of {@link ClickHouseResponseSummary} one for each execution
* @throws IllegalArgumentException if no named parameter in the query, or
* templates or params is null or empty
*/
static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode server, String sql,
ClickHouseValue[] templates, Object[]... params) {
Expand Down Expand Up @@ -455,6 +474,8 @@ static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode se
* @param sql non-null SQL query
* @param params non-null stringified parameters
* @return list of {@link ClickHouseResponseSummary} one for each execution
* @throws IllegalArgumentException if any of server, sql, or params is null; or
* no named parameter in the query
*/
static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode server, String sql,
String[][] params) {
Expand Down Expand Up @@ -493,13 +514,13 @@ static CompletableFuture<List<ClickHouseResponseSummary>> send(ClickHouseNode se
}

/**
* Tests if the given server is healthy or not. Unlike other methods, it's a
* Tests if the given server is alive or not. Unlike other methods, it's a
* synchronous call with minimum overhead(e.g. tiny buffer, no compression and
* no deserialization etc).
*
* @param server server to test
* @param timeout timeout in millisecond
* @return true if the server is healthy; false otherwise
* @return true if the server is alive; false otherwise
*/
static boolean test(ClickHouseNode server, int timeout) {
if (server != null) {
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
package com.clickhouse.client;

import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
Expand All @@ -9,12 +10,15 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;

/**
* A parameterized query is a parsed query with named parameters being extracted
* for substitution.
*/
public final class ClickHouseParameterizedQuery {
public final class ClickHouseParameterizedQuery implements Serializable {
private static final long serialVersionUID = 8108887349618342152L;

/**
* Substitute named parameters in given SQL.
*
Expand Down Expand Up @@ -97,11 +101,12 @@ public static ClickHouseParameterizedQuery of(String query) {
private final String originalQuery;
// 0 - from; 1 - to; 2 - parameter index(-1 means no parameter)
private final List<int[]> parts;

private int[] lastPart;
private String[] names;

private ClickHouseParameterizedQuery(String query) {
originalQuery = ClickHouseChecker.nonEmpty(query, "query");
originalQuery = ClickHouseChecker.nonBlank(query, "query");

parts = new LinkedList<>();
lastPart = null;
Expand Down Expand Up @@ -396,4 +401,30 @@ public List<String[]> getQueryParts() {
public boolean hasParameter() {
return names.length > 0;
}

@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + Arrays.hashCode(lastPart);
result = prime * result + Arrays.hashCode(names);
result = prime * result + ((originalQuery == null) ? 0 : originalQuery.hashCode());
result = prime * result + ((parts == null) ? 0 : parts.hashCode());
return result;
}

@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}

ClickHouseParameterizedQuery other = (ClickHouseParameterizedQuery) obj;
return Arrays.equals(lastPart, other.lastPart) && Arrays.equals(names, other.names)
&& Objects.equals(originalQuery, other.originalQuery) && ((parts.isEmpty() && other.parts.isEmpty())
|| Arrays.deepEquals(parts.toArray(new int[0][]), other.parts.toArray(new int[0][])));
}
}
Loading