Async reads for JDBC (#13196)

Async reads for JDBC:
Prevents JDBC timeouts on long queries by returning empty batches
when a batch fetch takes too long. Uses an async model to run the
result fetch concurrently with JDBC requests.

Fixed race condition in Druid's Avatica server-side handler
Fixed issue with no-user connections
This commit is contained in:
Paul Rogers 2022-10-18 11:40:57 -07:00 committed by GitHub
parent cc10350870
commit b34b4353f4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 663 additions and 331 deletions

View File

@ -43,7 +43,6 @@ import java.util.concurrent.TimeUnit;
*/ */
public class Execs public class Execs
{ {
/** /**
* Returns an ExecutorService which is terminated and shutdown from the beginning and not able to accept any tasks. * Returns an ExecutorService which is terminated and shutdown from the beginning and not able to accept any tasks.
*/ */

View File

@ -1888,6 +1888,7 @@ The Druid SQL server is configured through the following properties on the Broke
|`druid.sql.avatica.minRowsPerFrame`|Minimum acceptable value for the JDBC client `Statement.setFetchSize` method. The value for this property must greater than 0. If the JDBC client calls `Statement.setFetchSize` with a lesser value, Druid uses `minRowsPerFrame` instead. If `maxRowsPerFrame` is less than `minRowsPerFrame`, Druid uses the minimum value of the two. For handling queries which produce results with a large number of rows, you can increase this value to reduce the number of fetches required to completely transfer the result set.|100| |`druid.sql.avatica.minRowsPerFrame`|Minimum acceptable value for the JDBC client `Statement.setFetchSize` method. The value for this property must greater than 0. If the JDBC client calls `Statement.setFetchSize` with a lesser value, Druid uses `minRowsPerFrame` instead. If `maxRowsPerFrame` is less than `minRowsPerFrame`, Druid uses the minimum value of the two. For handling queries which produce results with a large number of rows, you can increase this value to reduce the number of fetches required to completely transfer the result set.|100|
|`druid.sql.avatica.maxStatementsPerConnection`|Maximum number of simultaneous open statements per Avatica client connection.|4| |`druid.sql.avatica.maxStatementsPerConnection`|Maximum number of simultaneous open statements per Avatica client connection.|4|
|`druid.sql.avatica.connectionIdleTimeout`|Avatica client connection idle timeout.|PT5M| |`druid.sql.avatica.connectionIdleTimeout`|Avatica client connection idle timeout.|PT5M|
|`druid.sql.avatica.fetchTimeoutMs`|Avatica fetch timeout, in milliseconds. When a request for the next batch of data takes longer than this time, Druid returns an empty result set, causing the client to poll again. This avoids HTTP timeouts for long-running queries. The default of 5 sec. is good for most cases. |5000|
|`druid.sql.http.enable`|Whether to enable JSON over HTTP querying at `/druid/v2/sql/`.|true| |`druid.sql.http.enable`|Whether to enable JSON over HTTP querying at `/druid/v2/sql/`.|true|
|`druid.sql.planner.maxTopNLimit`|Maximum threshold for a [TopN query](../querying/topnquery.md). Higher limits will be planned as [GroupBy queries](../querying/groupbyquery.md) instead.|100000| |`druid.sql.planner.maxTopNLimit`|Maximum threshold for a [TopN query](../querying/topnquery.md). Higher limits will be planned as [GroupBy queries](../querying/groupbyquery.md) instead.|100000|
|`druid.sql.planner.metadataRefreshPeriod`|Throttle for metadata refreshes.|PT1M| |`druid.sql.planner.metadataRefreshPeriod`|Throttle for metadata refreshes.|PT1M|

View File

@ -402,7 +402,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet implements Qu
HttpServletResponse response HttpServletResponse response
) throws ServletException, IOException ) throws ServletException, IOException
{ {
// Just call the superclass service method. Overriden in tests. // Just call the superclass service method. Overridden in tests.
super.service(request, response); super.service(request, response);
} }

View File

@ -247,9 +247,13 @@
<dependency> <dependency>
<groupId>org.mockito</groupId> <groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId> <artifactId>mockito-core</artifactId>
<version>${mockito.version}</version>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.jdbi</groupId>
<artifactId>jdbi</artifactId>
<scope>test</scope>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -27,6 +27,7 @@ import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PrepareResult; import org.apache.druid.sql.calcite.planner.PrepareResult;
@ -56,16 +57,19 @@ public abstract class AbstractDruidJdbcStatement implements Closeable
protected final String connectionId; protected final String connectionId;
protected final int statementId; protected final int statementId;
protected final ResultFetcherFactory fetcherFactory;
protected Throwable throwable; protected Throwable throwable;
protected DruidJdbcResultSet resultSet; protected DruidJdbcResultSet resultSet;
public AbstractDruidJdbcStatement( public AbstractDruidJdbcStatement(
final String connectionId, final String connectionId,
final int statementId final int statementId,
final ResultFetcherFactory fetcherFactory
) )
{ {
this.connectionId = Preconditions.checkNotNull(connectionId, "connectionId"); this.connectionId = Preconditions.checkNotNull(connectionId, "connectionId");
this.statementId = statementId; this.statementId = statementId;
this.fetcherFactory = fetcherFactory;
} }
protected static Meta.Signature createSignature( protected static Meta.Signature createSignature(

View File

@ -30,6 +30,7 @@ class AvaticaServerConfig
public static Period DEFAULT_CONNECTION_IDLE_TIMEOUT = new Period("PT5M"); public static Period DEFAULT_CONNECTION_IDLE_TIMEOUT = new Period("PT5M");
public static int DEFAULT_MIN_ROWS_PER_FRAME = 100; public static int DEFAULT_MIN_ROWS_PER_FRAME = 100;
public static int DEFAULT_MAX_ROWS_PER_FRAME = 5000; public static int DEFAULT_MAX_ROWS_PER_FRAME = 5000;
public static int DEFAULT_FETCH_TIMEOUT_MS = 5000;
@JsonProperty @JsonProperty
public int maxConnections = DEFAULT_MAX_CONNECTIONS; public int maxConnections = DEFAULT_MAX_CONNECTIONS;
@ -46,6 +47,17 @@ class AvaticaServerConfig
@JsonProperty @JsonProperty
public int maxRowsPerFrame = DEFAULT_MAX_ROWS_PER_FRAME; public int maxRowsPerFrame = DEFAULT_MAX_ROWS_PER_FRAME;
/**
* The maximum amount of time to wait per-fetch for the next result set.
* If a query takes longer than this amount of time, then the fetch will
* return 0 rows, without EOF, and the client will automatically try
* another fetch. The result is an async protocol that avoids network
* timeouts for long-running queries, especially those that take a long
* time to deliver a first batch of results.
*/
@JsonProperty
public int fetchTimeoutMs = DEFAULT_FETCH_TIMEOUT_MS;
public int getMaxConnections() public int getMaxConnections()
{ {
return maxConnections; return maxConnections;
@ -77,4 +89,9 @@ class AvaticaServerConfig
} }
return minRowsPerFrame; return minRowsPerFrame;
} }
public int getFetchTimeoutMs()
{
return fetchTimeoutMs;
}
} }

View File

@ -35,8 +35,8 @@ import java.io.IOException;
public class DruidAvaticaJsonHandler extends AvaticaJsonHandler public class DruidAvaticaJsonHandler extends AvaticaJsonHandler
{ {
public static final String AVATICA_PATH = "/druid/v2/sql/avatica/";
public static final String AVATICA_PATH_NO_TRAILING_SLASH = "/druid/v2/sql/avatica"; public static final String AVATICA_PATH_NO_TRAILING_SLASH = "/druid/v2/sql/avatica";
public static final String AVATICA_PATH = AVATICA_PATH_NO_TRAILING_SLASH + "/";
@Inject @Inject
public DruidAvaticaJsonHandler( public DruidAvaticaJsonHandler(

View File

@ -35,8 +35,8 @@ import java.io.IOException;
public class DruidAvaticaProtobufHandler extends AvaticaProtobufHandler public class DruidAvaticaProtobufHandler extends AvaticaProtobufHandler
{ {
public static final String AVATICA_PATH = "/druid/v2/sql/avatica-protobuf/";
public static final String AVATICA_PATH_NO_TRAILING_SLASH = "/druid/v2/sql/avatica-protobuf"; public static final String AVATICA_PATH_NO_TRAILING_SLASH = "/druid/v2/sql/avatica-protobuf";
public static final String AVATICA_PATH = AVATICA_PATH_NO_TRAILING_SLASH + "/";
@Inject @Inject
public DruidAvaticaProtobufHandler( public DruidAvaticaProtobufHandler(

View File

@ -24,8 +24,10 @@ import com.google.common.collect.ImmutableList;
import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.sql.PreparedStatement;
import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlQueryPlus;
import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import java.util.Collections; import java.util.Collections;
import java.util.Map; import java.util.Map;
@ -37,6 +39,11 @@ import java.util.concurrent.atomic.AtomicReference;
/** /**
* Connection tracking for {@link DruidMeta}. Thread-safe. * Connection tracking for {@link DruidMeta}. Thread-safe.
* <p>
* Lock is the instance itself. Used here to protect two members, and in
* other code when we must resolve the connection after resolving the statement.
* The lock prevents closing the connection concurrently with an operation on
* a statement for that connection.
*/ */
public class DruidConnection public class DruidConnection
{ {
@ -56,13 +63,12 @@ public class DruidConnection
private final AtomicInteger statementCounter = new AtomicInteger(); private final AtomicInteger statementCounter = new AtomicInteger();
private final AtomicReference<Future<?>> timeoutFuture = new AtomicReference<>(); private final AtomicReference<Future<?>> timeoutFuture = new AtomicReference<>();
// Typically synchronized by connectionLock, except in one case: the onClose function passed // Typically synchronized by this instance, except in one case: the onClose function passed
// into DruidStatements contained by the map. // into DruidStatements contained by the map.
@GuardedBy("connectionLock") @GuardedBy("this")
private final ConcurrentMap<Integer, AbstractDruidJdbcStatement> statements = new ConcurrentHashMap<>(); private final ConcurrentMap<Integer, AbstractDruidJdbcStatement> statements = new ConcurrentHashMap<>();
private final Object connectionLock = new Object();
@GuardedBy("connectionLock") @GuardedBy("this")
private boolean open = true; private boolean open = true;
public DruidConnection( public DruidConnection(
@ -93,13 +99,14 @@ public class DruidConnection
return userSecret; return userSecret;
} }
public DruidJdbcStatement createStatement( public synchronized DruidJdbcStatement createStatement(
final SqlStatementFactory sqlStatementFactory final SqlStatementFactory sqlStatementFactory,
final ResultFetcherFactory fetcherFactory
) )
{ {
final int statementId = statementCounter.incrementAndGet(); final int statementId = statementCounter.incrementAndGet();
synchronized (connectionLock) { synchronized (this) {
if (statements.containsKey(statementId)) { if (statements.containsKey(statementId)) {
// Will only happen if statementCounter rolls over before old statements are cleaned up. If this // Will only happen if statementCounter rolls over before old statements are cleaned up. If this
// ever happens then something fishy is going on, because we shouldn't have billions of statements. // ever happens then something fishy is going on, because we shouldn't have billions of statements.
@ -114,7 +121,9 @@ public class DruidConnection
final DruidJdbcStatement statement = new DruidJdbcStatement( final DruidJdbcStatement statement = new DruidJdbcStatement(
connectionId, connectionId,
statementId, statementId,
sqlStatementFactory sessionContext,
sqlStatementFactory,
fetcherFactory
); );
statements.put(statementId, statement); statements.put(statementId, statement);
@ -123,15 +132,16 @@ public class DruidConnection
} }
} }
public DruidJdbcPreparedStatement createPreparedStatement( public synchronized DruidJdbcPreparedStatement createPreparedStatement(
final SqlStatementFactory sqlStatementFactory, final SqlStatementFactory sqlStatementFactory,
final SqlQueryPlus sqlQueryPlus, final SqlQueryPlus sqlQueryPlus,
final long maxRowCount final long maxRowCount,
final ResultFetcherFactory fetcherFactory
) )
{ {
final int statementId = statementCounter.incrementAndGet(); final int statementId = statementCounter.incrementAndGet();
synchronized (connectionLock) { synchronized (this) {
if (statements.containsKey(statementId)) { if (statements.containsKey(statementId)) {
// Will only happen if statementCounter rolls over before old statements are cleaned up. If this // Will only happen if statementCounter rolls over before old statements are cleaned up. If this
// ever happens then something fishy is going on, because we shouldn't have billions of statements. // ever happens then something fishy is going on, because we shouldn't have billions of statements.
@ -143,11 +153,15 @@ public class DruidConnection
} }
@SuppressWarnings("GuardedBy") @SuppressWarnings("GuardedBy")
final PreparedStatement statement = sqlStatementFactory.preparedStatement(
sqlQueryPlus.withContext(sessionContext)
);
final DruidJdbcPreparedStatement jdbcStmt = new DruidJdbcPreparedStatement( final DruidJdbcPreparedStatement jdbcStmt = new DruidJdbcPreparedStatement(
connectionId, connectionId,
statementId, statementId,
sqlStatementFactory.preparedStatement(sqlQueryPlus), statement,
maxRowCount maxRowCount,
fetcherFactory
); );
statements.put(statementId, jdbcStmt); statements.put(statementId, jdbcStmt);
@ -156,17 +170,15 @@ public class DruidConnection
} }
} }
public AbstractDruidJdbcStatement getStatement(final int statementId) public synchronized AbstractDruidJdbcStatement getStatement(final int statementId)
{ {
synchronized (connectionLock) { return statements.get(statementId);
return statements.get(statementId);
}
} }
public void closeStatement(int statementId) public void closeStatement(int statementId)
{ {
AbstractDruidJdbcStatement stmt; AbstractDruidJdbcStatement stmt;
synchronized (connectionLock) { synchronized (this) {
stmt = statements.remove(statementId); stmt = statements.remove(statementId);
} }
if (stmt != null) { if (stmt != null) {
@ -180,34 +192,30 @@ public class DruidConnection
* *
* @return true if closed * @return true if closed
*/ */
public boolean closeIfEmpty() public synchronized boolean closeIfEmpty()
{ {
synchronized (connectionLock) { if (statements.isEmpty()) {
if (statements.isEmpty()) { close();
close(); return true;
return true; } else {
} else { return false;
return false;
}
} }
} }
public void close() public synchronized void close()
{ {
synchronized (connectionLock) { // Copy statements before iterating because statement.close() modifies it.
// Copy statements before iterating because statement.close() modifies it. for (AbstractDruidJdbcStatement statement : ImmutableList.copyOf(statements.values())) {
for (AbstractDruidJdbcStatement statement : ImmutableList.copyOf(statements.values())) { try {
try { statement.close();
statement.close(); }
} catch (Exception e) {
catch (Exception e) { LOG.warn("Connection [%s] failed to close statement [%s]!", connectionId, statement.getStatementId());
LOG.warn("Connection [%s] failed to close statement [%s]!", connectionId, statement.getStatementId());
}
} }
LOG.debug("Connection [%s] closed.", connectionId);
open = false;
} }
LOG.debug("Connection [%s] closed.", connectionId);
open = false;
} }
public DruidConnection sync(final Future<?> newTimeoutFuture) public DruidConnection sync(final Future<?> newTimeoutFuture)

View File

@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.ForbiddenException;
import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.DirectStatement;
import org.apache.druid.sql.PreparedStatement; import org.apache.druid.sql.PreparedStatement;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import org.apache.druid.sql.calcite.planner.PrepareResult; import org.apache.druid.sql.calcite.planner.PrepareResult;
import java.util.List; import java.util.List;
@ -50,10 +51,11 @@ public class DruidJdbcPreparedStatement extends AbstractDruidJdbcStatement
final String connectionId, final String connectionId,
final int statementId, final int statementId,
final PreparedStatement stmt, final PreparedStatement stmt,
final long maxRowCount final long maxRowCount,
final ResultFetcherFactory fetcherFactory
) )
{ {
super(connectionId, statementId); super(connectionId, statementId, fetcherFactory);
this.sqlStatement = stmt; this.sqlStatement = stmt;
this.maxRowCount = maxRowCount; this.maxRowCount = maxRowCount;
} }
@ -98,7 +100,7 @@ public class DruidJdbcPreparedStatement extends AbstractDruidJdbcStatement
closeResultSet(); closeResultSet();
try { try {
DirectStatement directStmt = sqlStatement.execute(parameters); DirectStatement directStmt = sqlStatement.execute(parameters);
resultSet = new DruidJdbcResultSet(this, directStmt, maxRowCount); resultSet = new DruidJdbcResultSet(this, directStmt, maxRowCount, fetcherFactory);
resultSet.execute(); resultSet.execute();
} }
// Failure to execute does not close the prepared statement. // Failure to execute does not close the prepared statement.

View File

@ -22,20 +22,27 @@ package org.apache.druid.sql.avatica;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.calcite.avatica.Meta; import org.apache.calcite.avatica.Meta;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.DirectStatement;
import java.io.Closeable; import java.io.Closeable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/** /**
* Druid's server-side representation of a JDBC result set. At most one * Druid's server-side representation of a JDBC result set. At most one
@ -59,6 +66,105 @@ import java.util.concurrent.ExecutorService;
*/ */
public class DruidJdbcResultSet implements Closeable public class DruidJdbcResultSet implements Closeable
{ {
private static final Logger LOG = new Logger(DruidJdbcResultSet.class);
/**
* Asynchronous result fetcher. JDBC operates via REST, which is subject to
* a timeout if a query takes too long to respond. Fortunately, JDBC uses a
* batched API, and is perfectly happy to get an empty batch. This class
* runs in a separate thread to fetch a batch. If the fetch takes too long,
* the JDBC request thread will time out waiting, will return an empty batch
* to the client, and will remember the fetch for use in the next fetch
* request. The result is that the time it takes to produce results for long
* running queries is decoupled from the HTTP timeout.
*/
public static class ResultFetcher implements Callable<Meta.Frame>
{
private final int limit;
private int batchSize;
private int offset;
private Yielder<Object[]> yielder;
public ResultFetcher(
final int limit,
final Yielder<Object[]> yielder
)
{
this.limit = limit;
this.yielder = yielder;
}
/**
* In an ideal world, the batch size would be a constructor parameter. But, JDBC,
* oddly, allows a different batch size per request. Hence, we set the size using
* this method before each fetch attempt.
*/
public void setBatchSize(int batchSize)
{
this.batchSize = batchSize;
}
/**
* Result is only valid between executions, which turns out to be
* the only time it is called.
*/
public int offset()
{
return offset;
}
/**
* Fetch the next batch up to the batch size or EOF. Return
* the resulting frame. Exceptions are handled by the executor
* framework.
*/
@Override
public Meta.Frame call()
{
Preconditions.checkState(batchSize > 0);
int rowCount = 0;
final int batchLimit = Math.min(limit - offset, batchSize);
final List<Object> rows = new ArrayList<>(batchLimit);
while (!yielder.isDone() && rowCount < batchLimit) {
rows.add(yielder.get());
yielder = yielder.next(null);
rowCount++;
}
final Meta.Frame result = new Meta.Frame(offset, yielder.isDone(), rows);
offset += rowCount;
return result;
}
}
/**
* Creates the result fetcher and holds config. Rather overkill for production,
* but handy for testing.
*/
public static class ResultFetcherFactory
{
final int fetchTimeoutMs;
public ResultFetcherFactory(int fetchTimeoutMs)
{
// To prevent server hammering, the timeout must be at least 1 second.
this.fetchTimeoutMs = Math.max(1000, fetchTimeoutMs);
}
public int fetchTimeoutMs()
{
return fetchTimeoutMs;
}
public ResultFetcher newFetcher(
final int limit,
final Yielder<Object[]> yielder
)
{
return new ResultFetcher(limit, yielder);
}
}
/** /**
* Query metrics can only be used within a single thread. Because results can * Query metrics can only be used within a single thread. Because results can
* be paginated into multiple JDBC frames (each frame being processed by a * be paginated into multiple JDBC frames (each frame being processed by a
@ -77,25 +183,46 @@ public class DruidJdbcResultSet implements Closeable
* https://github.com/apache/druid/pull/4288 * https://github.com/apache/druid/pull/4288
* https://github.com/apache/druid/pull/4415 * https://github.com/apache/druid/pull/4415
*/ */
private final ExecutorService yielderOpenCloseExecutor; private final ExecutorService queryExecutor;
private final DirectStatement stmt; private final DirectStatement stmt;
private final long maxRowCount; private final long maxRowCount;
private final ResultFetcherFactory fetcherFactory;
private State state = State.NEW; private State state = State.NEW;
private Meta.Signature signature; private Meta.Signature signature;
private Yielder<Object[]> yielder;
private int offset; /**
* The fetcher which reads batches of rows. Holds onto the yielder for a
* query. Maintains the current read offset.
*/
private ResultFetcher fetcher;
/**
* Future for a fetch that timed out waiting, and should be used again on
* the next fetch request.
*/
private Future<Meta.Frame> fetchFuture;
/**
* Cached version of the read offset in case the caller asks for the offset
* concurrently with a fetch which may update its own offset. This offset
* is that for the last batch that the client fetched: the fetcher itself
* may be moving to a new offset.
*/
private int nextFetchOffset;
public DruidJdbcResultSet( public DruidJdbcResultSet(
final AbstractDruidJdbcStatement jdbcStatement, final AbstractDruidJdbcStatement jdbcStatement,
final DirectStatement stmt, final DirectStatement stmt,
final long maxRowCount final long maxRowCount,
final ResultFetcherFactory fetcherFactory
) )
{ {
this.stmt = stmt; this.stmt = stmt;
this.maxRowCount = maxRowCount; this.maxRowCount = maxRowCount;
this.yielderOpenCloseExecutor = Execs.singleThreaded( this.fetcherFactory = fetcherFactory;
this.queryExecutor = Execs.singleThreaded(
StringUtils.format( StringUtils.format(
"JDBCYielderOpenCloseExecutor-connection-%s-statement-%d", "JDBCQueryExecutor-connection-%s-statement-%d",
StringUtils.encodeForFormat(jdbcStatement.getConnectionId()), StringUtils.encodeForFormat(jdbcStatement.getConnectionId()),
jdbcStatement.getStatementId() jdbcStatement.getStatementId()
) )
@ -107,19 +234,22 @@ public class DruidJdbcResultSet implements Closeable
ensure(State.NEW); ensure(State.NEW);
try { try {
state = State.RUNNING; state = State.RUNNING;
final Sequence<Object[]> baseSequence = yielderOpenCloseExecutor.submit(stmt::execute).get().getResults();
// We can't apply limits greater than Integer.MAX_VALUE, ignore them. // Execute the first step: plan the query and return a sequence to use
final Sequence<Object[]> retSequence = // to get values.
maxRowCount >= 0 && maxRowCount <= Integer.MAX_VALUE final Sequence<Object[]> sequence = queryExecutor.submit(stmt::execute).get().getResults();
? baseSequence.limit((int) maxRowCount)
: baseSequence;
yielder = Yielders.each(retSequence); // Subsequent fetch steps are done via the async "fetcher".
fetcher = fetcherFactory.newFetcher(
// We can't apply limits greater than Integer.MAX_VALUE, ignore them.
maxRowCount >= 0 && maxRowCount <= Integer.MAX_VALUE ? (int) maxRowCount : Integer.MAX_VALUE,
Yielders.each(sequence)
);
signature = AbstractDruidJdbcStatement.createSignature( signature = AbstractDruidJdbcStatement.createSignature(
stmt.prepareResult(), stmt.prepareResult(),
stmt.query().sql() stmt.query().sql()
); );
LOG.debug("Opened result set [%s]", stmt.sqlQueryId());
} }
catch (ExecutionException e) { catch (ExecutionException e) {
throw closeAndPropagateThrowable(e.getCause()); throw closeAndPropagateThrowable(e.getCause());
@ -143,34 +273,61 @@ public class DruidJdbcResultSet implements Closeable
public synchronized Meta.Frame nextFrame(final long fetchOffset, final int fetchMaxRowCount) public synchronized Meta.Frame nextFrame(final long fetchOffset, final int fetchMaxRowCount)
{ {
ensure(State.RUNNING, State.DONE); ensure(State.RUNNING, State.DONE);
Preconditions.checkState(fetchOffset == offset, "fetchOffset [%,d] != offset [%,d]", fetchOffset, offset); if (fetchOffset != nextFetchOffset) {
throw new IAE(
"Druid can only fetch forward. Requested offset of %,d != current offset %,d",
fetchOffset,
nextFetchOffset
);
}
if (state == State.DONE) { if (state == State.DONE) {
return new Meta.Frame(fetchOffset, true, Collections.emptyList()); LOG.debug("EOF at offset %,d for result set [%s]", fetchOffset, stmt.sqlQueryId());
return new Meta.Frame(fetcher.offset(), true, Collections.emptyList());
} }
final Future<Meta.Frame> future;
if (fetchFuture == null) {
// Not waiting on a batch. Request one now.
fetcher.setBatchSize(fetchMaxRowCount);
future = queryExecutor.submit(fetcher);
} else {
// Last batch took too long. Continue waiting for it.
future = fetchFuture;
fetchFuture = null;
}
try { try {
final List<Object> rows = new ArrayList<>(); Meta.Frame result = future.get(fetcherFactory.fetchTimeoutMs(), TimeUnit.MILLISECONDS);
while (!yielder.isDone() && (fetchMaxRowCount < 0 || offset < fetchOffset + fetchMaxRowCount)) { LOG.debug("Fetched batch at offset %,d for result set [%s]", fetchOffset, stmt.sqlQueryId());
rows.add(yielder.get()); if (result.done) {
yielder = yielder.next(null);
offset++;
}
if (yielder.isDone()) {
state = State.DONE; state = State.DONE;
} }
nextFetchOffset = fetcher.offset;
return new Meta.Frame(fetchOffset, state == State.DONE, rows); return result;
} }
catch (Throwable t) { catch (CancellationException | InterruptedException e) {
throw closeAndPropagateThrowable(t); // Consider this a failure.
throw closeAndPropagateThrowable(e);
}
catch (ExecutionException e) {
// Fetch threw an error. Unwrap it.
throw closeAndPropagateThrowable(e.getCause());
}
catch (TimeoutException e) {
LOG.debug("Timeout of batch at offset %,d for result set [%s]", fetchOffset, stmt.sqlQueryId());
fetchFuture = future;
// Wait timed out. Return 0 rows: the client will try again later.
// We'll wait again on this same fetch next time.
// Note that when the next fetch request comes, it will use the batch
// size set here: any change in size will be ignored for the in-flight batch.
// Changing batch size mid-query is an odd case: it will probably never happen.
return new Meta.Frame(nextFetchOffset, false, Collections.emptyList());
} }
} }
public synchronized long getCurrentOffset() public synchronized long getCurrentOffset()
{ {
ensure(State.RUNNING, State.DONE); ensure(State.RUNNING, State.DONE);
return offset; return fetcher.offset;
} }
@GuardedBy("this") @GuardedBy("this")
@ -215,14 +372,28 @@ public class DruidJdbcResultSet implements Closeable
if (state == State.CLOSED || state == State.FAILED) { if (state == State.CLOSED || state == State.FAILED) {
return; return;
} }
LOG.debug("Closing result set [%s]", stmt.sqlQueryId());
state = State.CLOSED; state = State.CLOSED;
try { try {
if (yielder != null) { // If a fetch is in progress, wait for it to complete.
Yielder<Object[]> theYielder = this.yielder; if (fetchFuture != null) {
this.yielder = null; try {
fetchFuture.cancel(true);
fetchFuture.get();
}
catch (Exception e) {
// Ignore, we're shutting down anyway.
}
finally {
fetchFuture = null;
}
}
if (fetcher != null) {
Yielder<Object[]> theYielder = fetcher.yielder;
fetcher = null;
// Put the close last, so any exceptions it throws are after we did the other cleanup above. // Put the close last, so any exceptions it throws are after we did the other cleanup above.
yielderOpenCloseExecutor.submit( queryExecutor.submit(
() -> { () -> {
theYielder.close(); theYielder.close();
// makes this a Callable instead of Runnable so we don't need to catch exceptions inside the lambda // makes this a Callable instead of Runnable so we don't need to catch exceptions inside the lambda
@ -230,7 +401,7 @@ public class DruidJdbcResultSet implements Closeable
} }
).get(); ).get();
yielderOpenCloseExecutor.shutdownNow(); queryExecutor.shutdownNow();
} }
} }
catch (RuntimeException e) { catch (RuntimeException e) {

View File

@ -24,6 +24,9 @@ import org.apache.calcite.avatica.Meta;
import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.DirectStatement;
import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlQueryPlus;
import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import java.util.Map;
/** /**
* Represents Druid's version of the JDBC {@code Statement} class: * Represents Druid's version of the JDBC {@code Statement} class:
@ -36,22 +39,27 @@ import org.apache.druid.sql.SqlStatementFactory;
public class DruidJdbcStatement extends AbstractDruidJdbcStatement public class DruidJdbcStatement extends AbstractDruidJdbcStatement
{ {
private final SqlStatementFactory lifecycleFactory; private final SqlStatementFactory lifecycleFactory;
protected final Map<String, Object> queryContext;
public DruidJdbcStatement( public DruidJdbcStatement(
final String connectionId, final String connectionId,
final int statementId, final int statementId,
final SqlStatementFactory lifecycleFactory final Map<String, Object> queryContext,
final SqlStatementFactory lifecycleFactory,
final ResultFetcherFactory fetcherFactory
) )
{ {
super(connectionId, statementId); super(connectionId, statementId, fetcherFactory);
this.queryContext = queryContext;
this.lifecycleFactory = Preconditions.checkNotNull(lifecycleFactory, "lifecycleFactory"); this.lifecycleFactory = Preconditions.checkNotNull(lifecycleFactory, "lifecycleFactory");
} }
public synchronized void execute(SqlQueryPlus queryPlus, long maxRowCount) public synchronized void execute(SqlQueryPlus queryPlus, long maxRowCount)
{ {
closeResultSet(); closeResultSet();
queryPlus = queryPlus.withContext(queryContext);
DirectStatement stmt = lifecycleFactory.directStatement(queryPlus); DirectStatement stmt = lifecycleFactory.directStatement(queryPlus);
resultSet = new DruidJdbcResultSet(this, stmt, Long.MAX_VALUE); resultSet = new DruidJdbcResultSet(this, stmt, Long.MAX_VALUE, fetcherFactory);
try { try {
resultSet.execute(); resultSet.execute();
} }

View File

@ -49,6 +49,7 @@ import org.apache.druid.server.security.AuthenticatorMapper;
import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.ForbiddenException;
import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlQueryPlus;
import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.Calcites;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -95,7 +96,20 @@ public class DruidMeta extends MetaImpl
*/ */
public static <T extends Throwable> T logFailure(T error) public static <T extends Throwable> T logFailure(T error)
{ {
logFailure(error, error.getMessage()); if (error instanceof NoSuchConnectionException) {
NoSuchConnectionException ex = (NoSuchConnectionException) error;
logFailure(error, "No such connection: %s", ex.getConnectionId());
} else if (error instanceof NoSuchStatementException) {
NoSuchStatementException ex = (NoSuchStatementException) error;
logFailure(
error,
"No such statement: %s, %d",
ex.getStatementHandle().connectionId,
ex.getStatementHandle().id
);
} else {
logFailure(error, error.getMessage());
}
return error; return error;
} }
@ -115,6 +129,7 @@ public class DruidMeta extends MetaImpl
private final AvaticaServerConfig config; private final AvaticaServerConfig config;
private final List<Authenticator> authenticators; private final List<Authenticator> authenticators;
private final ErrorHandler errorHandler; private final ErrorHandler errorHandler;
private final ResultFetcherFactory fetcherFactory;
/** /**
* Tracks logical connections. * Tracks logical connections.
@ -145,7 +160,8 @@ public class DruidMeta extends MetaImpl
.setDaemon(true) .setDaemon(true)
.build() .build()
), ),
authMapper.getAuthenticatorChain() authMapper.getAuthenticatorChain(),
new ResultFetcherFactory(config.getFetchTimeoutMs())
); );
} }
@ -154,7 +170,8 @@ public class DruidMeta extends MetaImpl
final AvaticaServerConfig config, final AvaticaServerConfig config,
final ErrorHandler errorHandler, final ErrorHandler errorHandler,
final ScheduledExecutorService exec, final ScheduledExecutorService exec,
final List<Authenticator> authenticators final List<Authenticator> authenticators,
final ResultFetcherFactory fetcherFactory
) )
{ {
super(null); super(null);
@ -163,6 +180,7 @@ public class DruidMeta extends MetaImpl
this.errorHandler = errorHandler; this.errorHandler = errorHandler;
this.exec = exec; this.exec = exec;
this.authenticators = authenticators; this.authenticators = authenticators;
this.fetcherFactory = fetcherFactory;
} }
@Override @Override
@ -188,11 +206,6 @@ public class DruidMeta extends MetaImpl
try { try {
openDruidConnection(ch.id, secret, contextMap); openDruidConnection(ch.id, secret, contextMap);
} }
catch (NoSuchConnectionException e) {
// Avoid sanitizing Avatica specific exceptions so that the Avatica code
// can rely on them to handle issues in a JDBC-specific way.
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -208,9 +221,6 @@ public class DruidMeta extends MetaImpl
druidConnection.close(); druidConnection.close();
} }
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -224,9 +234,6 @@ public class DruidMeta extends MetaImpl
getDruidConnection(ch.id); getDruidConnection(ch.id);
return connProps; return connProps;
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -241,12 +248,10 @@ public class DruidMeta extends MetaImpl
public StatementHandle createStatement(final ConnectionHandle ch) public StatementHandle createStatement(final ConnectionHandle ch)
{ {
try { try {
final DruidJdbcStatement druidStatement = getDruidConnection(ch.id).createStatement(sqlStatementFactory); final DruidJdbcStatement druidStatement = getDruidConnection(ch.id)
.createStatement(sqlStatementFactory, fetcherFactory);
return new StatementHandle(ch.id, druidStatement.getStatementId(), null); return new StatementHandle(ch.id, druidStatement.getStatementId(), null);
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -275,15 +280,13 @@ public class DruidMeta extends MetaImpl
final DruidJdbcPreparedStatement stmt = getDruidConnection(ch.id).createPreparedStatement( final DruidJdbcPreparedStatement stmt = getDruidConnection(ch.id).createPreparedStatement(
sqlStatementFactory, sqlStatementFactory,
sqlReq, sqlReq,
maxRowCount maxRowCount,
fetcherFactory
); );
stmt.prepare(); stmt.prepare();
LOG.debug("Successfully prepared statement [%s] for execution", stmt.getStatementId()); LOG.debug("Successfully prepared statement [%s] for execution", stmt.getStatementId());
return new StatementHandle(ch.id, stmt.getStatementId(), stmt.getSignature()); return new StatementHandle(ch.id, stmt.getStatementId(), stmt.getSignature());
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -315,7 +318,7 @@ public class DruidMeta extends MetaImpl
} }
/** /**
* Prepares and executes a JDBC {@code Statement} * Prepares and executes a JDBC {@code Statement}.
*/ */
@Override @Override
public ExecuteResult prepareAndExecute( public ExecuteResult prepareAndExecute(
@ -324,26 +327,25 @@ public class DruidMeta extends MetaImpl
final long maxRowCount, final long maxRowCount,
final int maxRowsInFirstFrame, final int maxRowsInFirstFrame,
final PrepareCallback callback final PrepareCallback callback
) throws NoSuchStatementException )
{ {
try { try {
// Ignore "callback", this class is designed for use with LocalService which doesn't use it. // Ignore "callback", this class is designed for use with LocalService which doesn't use it.
final DruidJdbcStatement druidStatement = getDruidStatement(statement, DruidJdbcStatement.class); final DruidJdbcStatement druidStatement = getDruidStatement(statement, DruidJdbcStatement.class);
final DruidConnection druidConnection = getDruidConnection(statement.connectionId); final DruidConnection druidConnection = getDruidConnection(statement.connectionId);
AuthenticationResult authenticationResult = doAuthenticate(druidConnection);
SqlQueryPlus sqlRequest = SqlQueryPlus.builder(sql) // This method is called directly from the Avatica server: it does not go
.auth(authenticationResult) // through the connection first. We must lock the connection here to prevent race conditions.
.context(druidConnection.sessionContext()) synchronized (druidConnection) {
.build(); final AuthenticationResult authenticationResult = doAuthenticate(druidConnection);
druidStatement.execute(sqlRequest, maxRowCount); final SqlQueryPlus sqlRequest = SqlQueryPlus.builder(sql)
ExecuteResult result = doFetch(druidStatement, maxRowsInFirstFrame); .auth(authenticationResult)
LOG.debug("Successfully prepared statement [%s] and started execution", druidStatement.getStatementId()); .build();
return result; druidStatement.execute(sqlRequest, maxRowCount);
} final ExecuteResult result = doFetch(druidStatement, maxRowsInFirstFrame);
// Cannot affect these exceptions as Avatica handles them. LOG.debug("Successfully prepared statement [%s] and started execution", druidStatement.getStatementId());
catch (NoSuchConnectionException | NoSuchStatementException e) { return result;
throw e; }
} }
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
@ -357,6 +359,15 @@ public class DruidMeta extends MetaImpl
*/ */
private RuntimeException mapException(Throwable t) private RuntimeException mapException(Throwable t)
{ {
// Don't sanitize or wrap Avatica exceptions: these exceptions
// are handled specially by Avatica to provide SQLState, Error Code
// and other JDBC-specific items.
if (t instanceof AvaticaRuntimeException) {
throw (AvaticaRuntimeException) t;
}
if (t instanceof NoSuchConnectionException) {
throw (NoSuchConnectionException) t;
}
// BasicSecurityAuthenticationException is not visible here. // BasicSecurityAuthenticationException is not visible here.
String className = t.getClass().getSimpleName(); String className = t.getClass().getSimpleName();
if (t instanceof ForbiddenException || if (t instanceof ForbiddenException ||
@ -365,7 +376,8 @@ public class DruidMeta extends MetaImpl
t.getMessage(), t.getMessage(),
ErrorResponse.UNAUTHORIZED_ERROR_CODE, ErrorResponse.UNAUTHORIZED_ERROR_CODE,
ErrorResponse.UNAUTHORIZED_SQL_STATE, ErrorResponse.UNAUTHORIZED_SQL_STATE,
AvaticaSeverity.ERROR); AvaticaSeverity.ERROR
);
} }
// Let Avatica do its default mapping. // Let Avatica do its default mapping.
@ -425,9 +437,6 @@ public class DruidMeta extends MetaImpl
LOG.debug("Fetching next frame from offset %,d with %,d rows for statement [%s]", offset, maxRows, statement.id); LOG.debug("Fetching next frame from offset %,d with %,d rows for statement [%s]", offset, maxRows, statement.id);
return getDruidStatement(statement, AbstractDruidJdbcStatement.class).nextFrame(offset, maxRows); return getDruidStatement(statement, AbstractDruidJdbcStatement.class).nextFrame(offset, maxRows);
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -450,7 +459,7 @@ public class DruidMeta extends MetaImpl
final StatementHandle statement, final StatementHandle statement,
final List<TypedValue> parameterValues, final List<TypedValue> parameterValues,
final int maxRowsInFirstFrame final int maxRowsInFirstFrame
) throws NoSuchStatementException )
{ {
try { try {
final DruidJdbcPreparedStatement druidStatement = final DruidJdbcPreparedStatement druidStatement =
@ -462,9 +471,6 @@ public class DruidMeta extends MetaImpl
druidStatement.getStatementId()); druidStatement.getStatementId());
return result; return result;
} }
catch (NoSuchStatementException | NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -493,9 +499,6 @@ public class DruidMeta extends MetaImpl
druidConnection.closeStatement(h.id); druidConnection.closeStatement(h.id);
} }
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -506,7 +509,7 @@ public class DruidMeta extends MetaImpl
final StatementHandle sh, final StatementHandle sh,
final QueryState state, final QueryState state,
final long offset final long offset
) throws NoSuchStatementException )
{ {
try { try {
final AbstractDruidJdbcStatement druidStatement = getDruidStatement(sh, AbstractDruidJdbcStatement.class); final AbstractDruidJdbcStatement druidStatement = getDruidStatement(sh, AbstractDruidJdbcStatement.class);
@ -521,9 +524,6 @@ public class DruidMeta extends MetaImpl
} }
return !isDone; return !isDone;
} }
catch (NoSuchStatementException | NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -560,9 +560,6 @@ public class DruidMeta extends MetaImpl
return sqlResultSet(ch, sql); return sqlResultSet(ch, sql);
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -597,9 +594,6 @@ public class DruidMeta extends MetaImpl
return sqlResultSet(ch, sql); return sqlResultSet(ch, sql);
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -656,9 +650,6 @@ public class DruidMeta extends MetaImpl
return sqlResultSet(ch, sql); return sqlResultSet(ch, sql);
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -726,9 +717,6 @@ public class DruidMeta extends MetaImpl
return sqlResultSet(ch, sql); return sqlResultSet(ch, sql);
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -747,9 +735,6 @@ public class DruidMeta extends MetaImpl
return sqlResultSet(ch, sql); return sqlResultSet(ch, sql);
} }
catch (NoSuchConnectionException e) {
throw e;
}
catch (Throwable t) { catch (Throwable t) {
throw mapException(t); throw mapException(t);
} }
@ -851,7 +836,7 @@ public class DruidMeta extends MetaImpl
return connection.sync( return connection.sync(
exec.schedule( exec.schedule(
() -> { () -> {
LOG.debug("Connection[%s] timed out.", connectionId); LOG.debug("Connection [%s] timed out.", connectionId);
closeConnection(new ConnectionHandle(connectionId)); closeConnection(new ConnectionHandle(connectionId));
}, },
new Interval(DateTimes.nowUtc(), config.getConnectionIdleTimeout()).toDurationMillis(), new Interval(DateTimes.nowUtc(), config.getConnectionIdleTimeout()).toDurationMillis(),

View File

@ -47,6 +47,7 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.math.expr.ExprMacroTable;
@ -67,6 +68,9 @@ import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthenticatorMapper; import org.apache.druid.server.security.AuthenticatorMapper;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.Escalator; import org.apache.druid.server.security.Escalator;
import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcher;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager; import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable; import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
@ -90,11 +94,12 @@ import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder; import org.junit.rules.TemporaryFolder;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.ResultIterator;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress;
import java.sql.Array; import java.sql.Array;
import java.sql.Connection; import java.sql.Connection;
import java.sql.DatabaseMetaData; import java.sql.DatabaseMetaData;
@ -117,30 +122,27 @@ import java.util.Set;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadLocalRandom;
/** /**
* Tests the Avatica-based JDBC implementation using JSON serialization. See * Tests the Avatica-based JDBC implementation using JSON serialization. See
* {@link DruidAvaticaProtobufHandlerTest} for a subclass which runs * {@link DruidAvaticaProtobufHandlerTest} for a subclass which runs
* this same set of tests using Protobuf serialization. * this same set of tests using Protobuf serialization.
* To run this in an IDE, set {@code -Duser.timezone=UTC}.
*/ */
public class DruidAvaticaHandlerTest extends CalciteTestBase public class DruidAvaticaHandlerTest extends CalciteTestBase
{ {
private static final AvaticaServerConfig AVATICA_CONFIG = new AvaticaServerConfig() private static final int CONNECTION_LIMIT = 4;
{ private static final int STATEMENT_LIMIT = 4;
@Override
public int getMaxConnections() private static final AvaticaServerConfig AVATICA_CONFIG;
{
// This must match the number of Connection objects created in testTooManyStatements() static {
return 4; AVATICA_CONFIG = new AvaticaServerConfig();
} // This must match the number of Connection objects created in testTooManyStatements()
AVATICA_CONFIG.maxConnections = CONNECTION_LIMIT;
AVATICA_CONFIG.maxStatementsPerConnection = STATEMENT_LIMIT;
}
@Override
public int getMaxStatementsPerConnection()
{
return 4;
}
};
private static final String DUMMY_SQL_QUERY_ID = "dummy"; private static final String DUMMY_SQL_QUERY_ID = "dummy";
private static QueryRunnerFactoryConglomerate conglomerate; private static QueryRunnerFactoryConglomerate conglomerate;
@ -162,35 +164,99 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
resourceCloser.close(); resourceCloser.close();
} }
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Rule @Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder(); public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule @Rule
public QueryLogHook queryLogHook = QueryLogHook.create(); public QueryLogHook queryLogHook = QueryLogHook.create();
private final PlannerConfig plannerConfig = new PlannerConfig();
private final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
private final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
private SpecificSegmentsQuerySegmentWalker walker; private SpecificSegmentsQuerySegmentWalker walker;
private Server server; private ServerWrapper server;
private Connection client; private Connection client;
private Connection clientNoTrailingSlash; private Connection clientNoTrailingSlash;
private Connection superuserClient; private Connection superuserClient;
private Connection clientLosAngeles; private Connection clientLosAngeles;
private DruidMeta druidMeta;
private String url;
private Injector injector; private Injector injector;
private TestRequestLogger testRequestLogger; private TestRequestLogger testRequestLogger;
private DruidSchemaCatalog makeRootSchema()
{
return CalciteTests.createMockRootSchema(
conglomerate,
walker,
plannerConfig,
CalciteTests.TEST_AUTHORIZER_MAPPER
);
}
private class ServerWrapper
{
final DruidMeta druidMeta;
final Server server;
final String url;
ServerWrapper(final DruidMeta druidMeta) throws Exception
{
this.druidMeta = druidMeta;
server = new Server(0);
server.setHandler(getAvaticaHandler(druidMeta));
server.start();
url = StringUtils.format(
"jdbc:avatica:remote:url=%s%s",
server.getURI().toString(),
StringUtils.maybeRemoveLeadingSlash(getJdbcUrlTail())
);
}
public Connection getConnection(String user, String password) throws SQLException
{
return DriverManager.getConnection(url, user, password);
}
public Connection getUserConnection() throws SQLException
{
return getConnection("regularUser", "druid");
}
// Note: though the URL-only form is OK in general, but it will cause tests
// to crash as the mock auth test code needs the user name.
// Use getUserConnection() instead, or create a URL that includes the
// user name and password.
//public Connection getConnection() throws SQLException
//{
// return DriverManager.getConnection(url);
//}
public void close() throws Exception
{
druidMeta.closeAllConnections();
server.stop();
}
}
protected String getJdbcUrlTail()
{
return DruidAvaticaJsonHandler.AVATICA_PATH;
}
// Default implementation is for JSON to allow debugging of tests.
protected AbstractAvaticaHandler getAvaticaHandler(final DruidMeta druidMeta)
{
return new DruidAvaticaJsonHandler(
druidMeta,
new DruidNode("dummy", "dummy", false, 1, null, true, false),
new AvaticaMonitor()
);
}
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
{ {
walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder()); walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder());
final PlannerConfig plannerConfig = new PlannerConfig(); final DruidSchemaCatalog rootSchema = makeRootSchema();
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
final DruidSchemaCatalog rootSchema =
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, CalciteTests.TEST_AUTHORIZER_MAPPER);
testRequestLogger = new TestRequestLogger(); testRequestLogger = new TestRequestLogger();
injector = new CoreInjectorBuilder(new StartupInjectorBuilder().build()) injector = new CoreInjectorBuilder(new StartupInjectorBuilder().build())
@ -227,37 +293,34 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
) )
.build(); .build();
druidMeta = injector.getInstance(DruidMeta.class); DruidMeta druidMeta = injector.getInstance(DruidMeta.class);
final AbstractAvaticaHandler handler = this.getAvaticaHandler(druidMeta); server = new ServerWrapper(druidMeta);
final int port = ThreadLocalRandom.current().nextInt(9999) + 10000; client = server.getUserConnection();
server = new Server(new InetSocketAddress("127.0.0.1", port)); superuserClient = server.getConnection(CalciteTests.TEST_SUPERUSER_NAME, "druid");
server.setHandler(handler); clientNoTrailingSlash = DriverManager.getConnection(StringUtils.maybeRemoveTrailingSlash(server.url), CalciteTests.TEST_SUPERUSER_NAME, "druid");
server.start();
url = this.getJdbcConnectionString(port);
client = DriverManager.getConnection(url, "regularUser", "druid");
superuserClient = DriverManager.getConnection(url, CalciteTests.TEST_SUPERUSER_NAME, "druid");
clientNoTrailingSlash = DriverManager.getConnection(StringUtils.maybeRemoveTrailingSlash(url), CalciteTests.TEST_SUPERUSER_NAME, "druid");
final Properties propertiesLosAngeles = new Properties(); final Properties propertiesLosAngeles = new Properties();
propertiesLosAngeles.setProperty("sqlTimeZone", "America/Los_Angeles"); propertiesLosAngeles.setProperty("sqlTimeZone", "America/Los_Angeles");
propertiesLosAngeles.setProperty("user", "regularUserLA"); propertiesLosAngeles.setProperty("user", "regularUserLA");
propertiesLosAngeles.setProperty(BaseQuery.SQL_QUERY_ID, DUMMY_SQL_QUERY_ID); propertiesLosAngeles.setProperty(BaseQuery.SQL_QUERY_ID, DUMMY_SQL_QUERY_ID);
clientLosAngeles = DriverManager.getConnection(url, propertiesLosAngeles); clientLosAngeles = DriverManager.getConnection(server.url, propertiesLosAngeles);
} }
@After @After
public void tearDown() throws Exception public void tearDown() throws Exception
{ {
client.close(); if (server != null) {
clientLosAngeles.close(); client.close();
clientNoTrailingSlash.close(); clientLosAngeles.close();
server.stop(); clientNoTrailingSlash.close();
server.close();
client = null;
clientLosAngeles = null;
clientNoTrailingSlash = null;
server = null;
}
walker.close(); walker.close();
walker = null; walker = null;
client = null;
clientLosAngeles = null;
clientNoTrailingSlash = null;
server = null;
} }
@Test @Test
@ -772,19 +835,21 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
@Test @Test
public void testTooManyStatements() throws SQLException public void testTooManyStatements() throws SQLException
{ {
for (int i = 0; i < 4; i++) { for (int i = 0; i < STATEMENT_LIMIT; i++) {
client.createStatement(); client.createStatement();
} }
expectedException.expect(AvaticaClientRuntimeException.class); AvaticaClientRuntimeException ex = Assert.assertThrows(
expectedException.expectMessage("Too many open statements, limit is 4"); AvaticaClientRuntimeException.class,
client.createStatement(); () -> client.createStatement()
);
Assert.assertTrue(ex.getMessage().contains("Too many open statements, limit is 4"));
} }
@Test @Test
public void testNotTooManyStatementsWhenYouCloseThem() throws SQLException public void testNotTooManyStatementsWhenYouCloseThem() throws SQLException
{ {
for (int i = 0; i < 10; i++) { for (int i = 0; i < STATEMENT_LIMIT * 2; i++) {
client.createStatement().close(); client.createStatement().close();
} }
} }
@ -863,7 +928,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
ImmutableList.of(ImmutableMap.of("cnt", 6L)), ImmutableList.of(ImmutableMap.of("cnt", 6L)),
getRows(resultSet) getRows(resultSet)
); );
druidMeta.closeAllConnections(); server.druidMeta.closeAllConnections();
} }
} }
@ -875,70 +940,72 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
superuserClient.createStatement(); superuserClient.createStatement();
clientNoTrailingSlash.createStatement(); clientNoTrailingSlash.createStatement();
expectedException.expect(AvaticaClientRuntimeException.class); AvaticaClientRuntimeException ex = Assert.assertThrows(
expectedException.expectMessage("Too many connections"); AvaticaClientRuntimeException.class,
() -> server.getUserConnection()
DriverManager.getConnection(url); );
Assert.assertTrue(ex.getMessage().contains("Too many connections"));
} }
@Test @Test
public void testNotTooManyConnectionsWhenTheyAreEmpty() throws SQLException public void testNotTooManyConnectionsWhenTheyAreClosed() throws SQLException
{ {
for (int i = 0; i < 4; i++) { for (int i = 0; i < CONNECTION_LIMIT * 2; i++) {
try (Connection connection = DriverManager.getConnection(url)) { try (Connection connection = server.getUserConnection()) {
} }
} }
} }
@Test
public void testConnectionsCloseStatements() throws SQLException
{
for (int i = 0; i < CONNECTION_LIMIT * 2; i++) {
try (Connection connection = server.getUserConnection()) {
// Note: NOT in a try-catch block. Let the connection close the statement
final Statement statement = connection.createStatement();
// Again, NOT in a try-catch block: let the statement close the
// result set.
final ResultSet resultSet = statement.executeQuery("SELECT COUNT(*) AS cnt FROM druid.foo");
Assert.assertTrue(resultSet.next());
}
}
}
private SqlStatementFactory makeStatementFactory()
{
return CalciteTests.createSqlStatementFactory(
CalciteTests.createMockSqlEngine(walker, conglomerate),
new PlannerFactory(
makeRootSchema(),
operatorTable,
macroTable,
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of())
)
);
}
@Test @Test
public void testMaxRowsPerFrame() throws Exception public void testMaxRowsPerFrame() throws Exception
{ {
final AvaticaServerConfig smallFrameConfig = new AvaticaServerConfig() final AvaticaServerConfig config = new AvaticaServerConfig();
{ config.maxConnections = 2;
@Override config.maxStatementsPerConnection = STATEMENT_LIMIT;
public int getMaxConnections() config.maxRowsPerFrame = 2;
{
return 2;
}
@Override
public int getMaxStatementsPerConnection()
{
return 4;
}
@Override
public int getMaxRowsPerFrame()
{
return 2;
}
};
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
final List<Meta.Frame> frames = new ArrayList<>(); final List<Meta.Frame> frames = new ArrayList<>();
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
DruidSchemaCatalog rootSchema =
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
DruidMeta smallFrameDruidMeta = new DruidMeta( DruidMeta smallFrameDruidMeta = new DruidMeta(
CalciteTests.createSqlStatementFactory( makeStatementFactory(),
CalciteTests.createMockSqlEngine(walker, conglomerate), config,
new PlannerFactory(
rootSchema,
operatorTable,
macroTable,
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of())
)
),
smallFrameConfig,
new ErrorHandler(new ServerConfig()), new ErrorHandler(new ServerConfig()),
exec, exec,
injector.getInstance(AuthenticatorMapper.class).getAuthenticatorChain() injector.getInstance(AuthenticatorMapper.class).getAuthenticatorChain(),
new ResultFetcherFactory(config.fetchTimeoutMs)
) )
{ {
@Override @Override
@ -955,13 +1022,8 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
} }
}; };
final AbstractAvaticaHandler handler = this.getAvaticaHandler(smallFrameDruidMeta); ServerWrapper server = new ServerWrapper(smallFrameDruidMeta);
final int port = ThreadLocalRandom.current().nextInt(9999) + 20000; Connection smallFrameClient = server.getUserConnection();
Server smallFrameServer = new Server(new InetSocketAddress("127.0.0.1", port));
smallFrameServer.setHandler(handler);
smallFrameServer.start();
String smallFrameUrl = this.getJdbcConnectionString(port);
Connection smallFrameClient = DriverManager.getConnection(smallFrameUrl, "regularUser", "druid");
final ResultSet resultSet = smallFrameClient.createStatement().executeQuery( final ResultSet resultSet = smallFrameClient.createStatement().executeQuery(
"SELECT dim1 FROM druid.foo" "SELECT dim1 FROM druid.foo"
@ -980,59 +1042,29 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
rows rows
); );
resultSet.close();
smallFrameClient.close();
exec.shutdown(); exec.shutdown();
server.close();
} }
@Test @Test
public void testMinRowsPerFrame() throws Exception public void testMinRowsPerFrame() throws Exception
{ {
final int minFetchSize = 1000; final AvaticaServerConfig config = new AvaticaServerConfig();
final AvaticaServerConfig smallFrameConfig = new AvaticaServerConfig() config.maxConnections = 2;
{ config.maxStatementsPerConnection = STATEMENT_LIMIT;
@Override config.minRowsPerFrame = 1000;
public int getMaxConnections()
{
return 2;
}
@Override
public int getMaxStatementsPerConnection()
{
return 4;
}
@Override
public int getMinRowsPerFrame()
{
return minFetchSize;
}
};
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
final List<Meta.Frame> frames = new ArrayList<>(); final List<Meta.Frame> frames = new ArrayList<>();
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
DruidSchemaCatalog rootSchema =
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
DruidMeta smallFrameDruidMeta = new DruidMeta( DruidMeta smallFrameDruidMeta = new DruidMeta(
CalciteTests.createSqlStatementFactory( makeStatementFactory(),
CalciteTests.createMockSqlEngine(walker, conglomerate), config,
new PlannerFactory(
rootSchema,
operatorTable,
macroTable,
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of())
)
),
smallFrameConfig,
new ErrorHandler(new ServerConfig()), new ErrorHandler(new ServerConfig()),
exec, exec,
injector.getInstance(AuthenticatorMapper.class).getAuthenticatorChain() injector.getInstance(AuthenticatorMapper.class).getAuthenticatorChain(),
new ResultFetcherFactory(config.fetchTimeoutMs)
) )
{ {
@Override @Override
@ -1043,20 +1075,15 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
) throws NoSuchStatementException, MissingResultsException ) throws NoSuchStatementException, MissingResultsException
{ {
// overriding fetch allows us to track how many frames are processed after the first frame, and also fetch size // overriding fetch allows us to track how many frames are processed after the first frame, and also fetch size
Assert.assertEquals(minFetchSize, fetchMaxRowCount); Assert.assertEquals(config.minRowsPerFrame, fetchMaxRowCount);
Frame frame = super.fetch(statement, offset, fetchMaxRowCount); Frame frame = super.fetch(statement, offset, fetchMaxRowCount);
frames.add(frame); frames.add(frame);
return frame; return frame;
} }
}; };
final AbstractAvaticaHandler handler = this.getAvaticaHandler(smallFrameDruidMeta); ServerWrapper server = new ServerWrapper(smallFrameDruidMeta);
final int port = ThreadLocalRandom.current().nextInt(9999) + 20000; Connection smallFrameClient = server.getUserConnection();
Server smallFrameServer = new Server(new InetSocketAddress("127.0.0.1", port));
smallFrameServer.setHandler(handler);
smallFrameServer.start();
String smallFrameUrl = this.getJdbcConnectionString(port);
Connection smallFrameClient = DriverManager.getConnection(smallFrameUrl, "regularUser", "druid");
// use a prepared statement because Avatica currently ignores fetchSize on the initial fetch of a Statement // use a prepared statement because Avatica currently ignores fetchSize on the initial fetch of a Statement
PreparedStatement statement = smallFrameClient.prepareStatement("SELECT dim1 FROM druid.foo"); PreparedStatement statement = smallFrameClient.prepareStatement("SELECT dim1 FROM druid.foo");
@ -1078,7 +1105,10 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
rows rows
); );
resultSet.close();
smallFrameClient.close();
exec.shutdown(); exec.shutdown();
server.close();
} }
@Test @Test
@ -1546,24 +1576,124 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Assert.fail("Test failed, did not get SQLException"); Assert.fail("Test failed, did not get SQLException");
} }
// Default implementation is for JSON to allow debugging of tests. private static class TestResultFetcher extends ResultFetcher
protected String getJdbcConnectionString(final int port)
{ {
return StringUtils.format( public TestResultFetcher(int limit, Yielder<Object[]> yielder)
"jdbc:avatica:remote:url=http://127.0.0.1:%d%s", {
port, super(limit, yielder);
DruidAvaticaJsonHandler.AVATICA_PATH }
);
@Override
public Meta.Frame call()
{
try {
if (offset() == 0) {
System.out.println("Taking a nap now...");
Thread.sleep(3000);
}
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
return super.call();
}
} }
// Default implementation is for JSON to allow debugging of tests. /**
protected AbstractAvaticaHandler getAvaticaHandler(final DruidMeta druidMeta) * Test the async aspect of the Avatica implementation. The fetch of the
* first batch takes 3 seconds (due to a sleep). However, the client will
* wait only 1 second. So, we should get ~3 empty batches before we get
* the first batch with rows.
*/
@Test
public void testAsync() throws Exception
{ {
return new DruidAvaticaJsonHandler( final AvaticaServerConfig config = new AvaticaServerConfig();
druidMeta, config.maxConnections = CONNECTION_LIMIT;
new DruidNode("dummy", "dummy", false, 1, null, true, false), config.maxStatementsPerConnection = STATEMENT_LIMIT;
new AvaticaMonitor() config.maxRowsPerFrame = 2;
); config.fetchTimeoutMs = 1000;
final List<Meta.Frame> frames = new ArrayList<>();
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
DruidMeta druidMeta = new DruidMeta(
makeStatementFactory(),
config,
new ErrorHandler(new ServerConfig()),
exec,
injector.getInstance(AuthenticatorMapper.class).getAuthenticatorChain(),
new ResultFetcherFactory(config.fetchTimeoutMs) {
@Override
public ResultFetcher newFetcher(
final int limit,
final Yielder<Object[]> yielder
)
{
return new TestResultFetcher(limit, yielder);
}
}
)
{
@Override
public Frame fetch(
final StatementHandle statement,
final long offset,
final int fetchMaxRowCount
) throws NoSuchStatementException, MissingResultsException
{
Frame frame = super.fetch(statement, offset, fetchMaxRowCount);
frames.add(frame);
return frame;
}
};
ServerWrapper server = new ServerWrapper(druidMeta);
try (Connection conn = server.getUserConnection()) {
// Test with plain JDBC
try (ResultSet resultSet = conn.createStatement().executeQuery(
"SELECT dim1 FROM druid.foo")) {
List<Map<String, Object>> rows = getRows(resultSet);
Assert.assertEquals(6, rows.size());
Assert.assertTrue(frames.size() > 3);
// There should be at least one empty frame due to timeout
Assert.assertFalse(frames.get(0).rows.iterator().hasNext());
}
}
testWithJDBI(server.url);
exec.shutdown();
server.close();
}
// Test the async feature using DBI, as used internally in Druid.
// Ensures that DBI knows how to handle empty batches (which should,
// in reality, but handled at the JDBC level below DBI.)
private void testWithJDBI(String baseUrl)
{
String url = baseUrl + "?user=regularUser&password=druid" + getJdbcUrlTail();
System.out.println(url);
DBI dbi = new DBI(url);
Handle handle = dbi.open();
try {
ResultIterator<Pair<Long, String>> iter = handle
.createQuery("SELECT __time, dim1 FROM druid.foo")
.map((index, row, ctx) -> new Pair<>(row.getLong(1), row.getString(2)))
.iterator();
int count = 0;
while (iter.hasNext()) {
Pair<Long, String> row = iter.next();
Assert.assertNotNull(row.lhs);
Assert.assertNotNull(row.rhs);
count++;
}
Assert.assertEquals(6, count);
}
finally {
handle.close();
}
} }
private static List<Map<String, Object>> getRows(final ResultSet resultSet) throws SQLException private static List<Map<String, Object>> getRows(final ResultSet resultSet) throws SQLException

View File

@ -26,11 +26,10 @@ import org.apache.druid.server.DruidNode;
public class DruidAvaticaProtobufHandlerTest extends DruidAvaticaHandlerTest public class DruidAvaticaProtobufHandlerTest extends DruidAvaticaHandlerTest
{ {
@Override @Override
protected String getJdbcConnectionString(final int port) protected String getJdbcUrlTail()
{ {
return StringUtils.format( return StringUtils.format(
"jdbc:avatica:remote:url=http://127.0.0.1:%d%s;serialization=protobuf", "%s;serialization=protobuf",
port,
DruidAvaticaProtobufHandler.AVATICA_PATH DruidAvaticaProtobufHandler.AVATICA_PATH
); );
} }

View File

@ -35,6 +35,7 @@ import org.apache.druid.server.security.AllowAllAuthenticator;
import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlQueryPlus;
import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.avatica.DruidJdbcResultSet.ResultFetcherFactory;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager; import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable; import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerConfig;
@ -138,7 +139,9 @@ public class DruidStatementTest extends CalciteTestBase
return new DruidJdbcStatement( return new DruidJdbcStatement(
"", "",
0, 0,
sqlStatementFactory Collections.emptyMap(),
sqlStatementFactory,
new ResultFetcherFactory(AvaticaServerConfig.DEFAULT_FETCH_TIMEOUT_MS)
); );
} }
@ -517,7 +520,8 @@ public class DruidStatementTest extends CalciteTestBase
"", "",
0, 0,
sqlStatementFactory.preparedStatement(queryPlus), sqlStatementFactory.preparedStatement(queryPlus),
Long.MAX_VALUE Long.MAX_VALUE,
new ResultFetcherFactory(AvaticaServerConfig.DEFAULT_FETCH_TIMEOUT_MS)
); );
} }

View File

@ -185,19 +185,19 @@ public class CalciteTests
public Authorizer getAuthorizer(String name) public Authorizer getAuthorizer(String name)
{ {
return (authenticationResult, resource, action) -> { return (authenticationResult, resource, action) -> {
if (authenticationResult.getIdentity().equals(TEST_SUPERUSER_NAME)) { if (TEST_SUPERUSER_NAME.equals(authenticationResult.getIdentity())) {
return Access.OK; return Access.OK;
} }
switch (resource.getType()) { switch (resource.getType()) {
case ResourceType.DATASOURCE: case ResourceType.DATASOURCE:
if (resource.getName().equals(FORBIDDEN_DATASOURCE)) { if (FORBIDDEN_DATASOURCE.equals(resource.getName())) {
return new Access(false); return new Access(false);
} else { } else {
return Access.OK; return Access.OK;
} }
case ResourceType.VIEW: case ResourceType.VIEW:
if (resource.getName().equals("forbiddenView")) { if ("forbiddenView".equals(resource.getName())) {
return new Access(false); return new Access(false);
} else { } else {
return Access.OK; return Access.OK;